You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ka...@apache.org on 2016/04/26 19:26:52 UTC

[01/13] mesos git commit: Added a realm parameter to `process::initialize` (libprocess).

Repository: mesos
Updated Branches:
  refs/heads/master c530deb30 -> dc8b9a4ab


Added a realm parameter to `process::initialize` (libprocess).

In order to enable authentication on libprocess-level
HTTP endpoints, this patch adds a parameter to
process::initialize which allows the authentication
realm of such endpoints to be set when libprocess is
initialized.

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


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

Branch: refs/heads/master
Commit: 3d781197873c8f388a9ed9152bb1b3bbbb686d32
Parents: c530deb
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:42:25 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:42:25 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/include/process/gtest.hpp   |  2 ++
 3rdparty/libprocess/include/process/process.hpp |  9 ++++++++-
 3rdparty/libprocess/src/process.cpp             | 15 ++++++++++++---
 3rdparty/libprocess/src/tests/main.cpp          |  2 +-
 4 files changed, 23 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3d781197/3rdparty/libprocess/include/process/gtest.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/gtest.hpp b/3rdparty/libprocess/include/process/gtest.hpp
index 3e08872..21f66ed 100644
--- a/3rdparty/libprocess/include/process/gtest.hpp
+++ b/3rdparty/libprocess/include/process/gtest.hpp
@@ -28,6 +28,8 @@
 
 namespace process {
 
+constexpr char DEFAULT_HTTP_AUTHENTICATION_REALM[] = "libprocess-realm";
+
 // 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).

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d781197/3rdparty/libprocess/include/process/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/process.hpp b/3rdparty/libprocess/include/process/process.hpp
index 77e9695..3a0d244 100644
--- a/3rdparty/libprocess/include/process/process.hpp
+++ b/3rdparty/libprocess/include/process/process.hpp
@@ -453,10 +453,17 @@ protected:
  * for it (e.g., a logging directory) via environment variables.
  *
  * @param delegate Process to receive root HTTP requests.
+ * @param authenticationRealm The authentication realm that libprocess-level
+ *     HTTP endpoints will be installed under, if any. If this realm is not
+ *     specified, endpoints will be installed without authentication.
+ * @return `true` if this was the first invocation of `process::initialize()`,
+ *     or `false` if it was not the first invocation.
  *
  * @see [glog](https://google-glog.googlecode.com/svn/trunk/doc/glog.html)
  */
-void initialize(const Option<std::string>& delegate = None());
+bool initialize(
+    const Option<std::string>& delegate = None(),
+    const Option<std::string>& authenticationRealm = None());
 
 
 /**

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d781197/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 8727eb2..d1d924c 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -775,7 +775,9 @@ void install(vector<Owned<FirewallRule>>&& rules)
 
 } // namespace firewall {
 
-void initialize(const Option<string>& delegate)
+bool initialize(
+    const Option<string>& delegate,
+    const Option<string>& authenticationRealm)
 {
   // TODO(benh): Return an error if attempting to initialize again
   // with a different delegate than originally specified.
@@ -797,7 +799,8 @@ void initialize(const Option<string>& delegate)
   // of initialization.  This is done because some methods called by
   // initialization will themselves call `process::initialize`.
   if (initialize_started.load() && initialize_complete.load()) {
-    return;
+    // Return `false` because `process::initialize()` was already called.
+    return false;
 
   } else {
     // NOTE: `compare_exchange_strong` needs an lvalue.
@@ -812,7 +815,9 @@ void initialize(const Option<string>& delegate)
     // initialization to complete.
     if (!initialize_started.compare_exchange_strong(expected, true)) {
       while (!initialize_complete.load());
-      return;
+
+      // Return `false` because `process::initialize()` was already called.
+      return false;
     }
   }
 
@@ -990,6 +995,10 @@ void initialize(const Option<string>& delegate)
 
   VLOG(1) << "libprocess is initialized on " << address() << " with "
           << num_worker_threads << " worker threads";
+
+  // Return `true` to indicate that this was the first invocation of
+  // `process::initialize()`.
+  return true;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3d781197/3rdparty/libprocess/src/tests/main.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/main.cpp b/3rdparty/libprocess/src/tests/main.cpp
index 78858a2..c2ce9bc 100644
--- a/3rdparty/libprocess/src/tests/main.cpp
+++ b/3rdparty/libprocess/src/tests/main.cpp
@@ -42,7 +42,7 @@ int main(int argc, char** argv)
   testing::InitGoogleMock(&argc, argv);
 
   // Initialize libprocess.
-  process::initialize();
+  process::initialize(None(), process::DEFAULT_HTTP_AUTHENTICATION_REALM);
 
   // Install GLOG's signal handler.
   google::InstallFailureSignalHandler();


[11/13] mesos git commit: Updated gperftools to version 2.5 (Mesos).

Posted by ka...@apache.org.
Updated gperftools to version 2.5 (Mesos).

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


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

Branch: refs/heads/master
Commit: 0956ce89bad62657e88affe233a6693632a5bc67
Parents: 5aec037
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:37 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:44:23 2016 -0400

----------------------------------------------------------------------
 src/python/native_common/ext_modules.py.in | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0956ce89/src/python/native_common/ext_modules.py.in
----------------------------------------------------------------------
diff --git a/src/python/native_common/ext_modules.py.in b/src/python/native_common/ext_modules.py.in
index ad98a57..4fd328f 100644
--- a/src/python/native_common/ext_modules.py.in
+++ b/src/python/native_common/ext_modules.py.in
@@ -42,7 +42,7 @@ def _create_module(module_name):
     # include them here (or more precisely, down where we actually include
     # libev.a and libprofiler.a).
     glog = os.path.join(libprocess, '3rdparty', 'glog-0.3.3')
-    gperftools = os.path.join(libprocess, '3rdparty', 'gperftools-2.0')
+    gperftools = os.path.join(libprocess, '3rdparty', 'gperftools-2.5')
     protobuf = os.path.join(libprocess, '3rdparty', 'protobuf-2.6.1')
 
     # Build the list of source files. Note that each source must be


[05/13] mesos git commit: Added a MetricsTest with authentication to libprocess.

Posted by ka...@apache.org.
Added a MetricsTest with authentication to libprocess.

The test `MetricsTest.SnapshotAuthenticationEnabled`
is added to the libprocess tests in this patch.

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


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

Branch: refs/heads/master
Commit: e0aaccf02356cfb89654e933122f2bf2be5d5fd6
Parents: 066fc4b
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:06 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:43:06 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/src/tests/metrics_tests.cpp | 95 ++++++++++++++++++--
 1 file changed, 87 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e0aaccf0/3rdparty/libprocess/src/tests/metrics_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/metrics_tests.cpp b/3rdparty/libprocess/src/tests/metrics_tests.cpp
index b84dc8d..1cda7b4 100644
--- a/3rdparty/libprocess/src/tests/metrics_tests.cpp
+++ b/3rdparty/libprocess/src/tests/metrics_tests.cpp
@@ -15,9 +15,11 @@
 #include <map>
 #include <string>
 
+#include <stout/base64.hpp>
 #include <stout/duration.hpp>
 #include <stout/gtest.hpp>
 
+#include <process/authenticator.hpp>
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gtest.hpp>
@@ -31,18 +33,24 @@
 #include <process/metrics/metrics.hpp>
 #include <process/metrics/timer.hpp>
 
+namespace authentication = process::http::authentication;
 namespace http = process::http;
 namespace metrics = process::metrics;
 
+using authentication::Authenticator;
+using authentication::BasicAuthenticator;
+
 using http::BadRequest;
 using http::OK;
 using http::Response;
+using http::Unauthorized;
 
 using metrics::Counter;
 using metrics::Gauge;
 using metrics::Timer;
 
 using process::Clock;
+using process::DEFAULT_HTTP_AUTHENTICATION_REALM;
 using process::Failure;
 using process::Future;
 using process::PID;
@@ -73,7 +81,36 @@ public:
 };
 
 
-TEST(MetricsTest, Counter)
+// TODO(greggomann): Move this into a base class in 'mesos.hpp'.
+class MetricsTest : public ::testing::Test
+{
+protected:
+  Future<Nothing> setAuthenticator(
+      const string& realm,
+      process::Owned<Authenticator> authenticator)
+  {
+    realms.insert(realm);
+
+    return authentication::setAuthenticator(realm, authenticator);
+  }
+
+  virtual void TearDown()
+  {
+    foreach (const string& realm, realms) {
+      // We need to wait in order to ensure that the operation
+      // completes before we leave TearDown. Otherwise, we may
+      // leak a mock object.
+      AWAIT_READY(authentication::unsetAuthenticator(realm));
+    }
+    realms.clear();
+  }
+
+private:
+  hashset<string> realms;
+};
+
+
+TEST_F(MetricsTest, Counter)
 {
   Counter counter("test/counter");
 
@@ -99,7 +136,7 @@ TEST(MetricsTest, Counter)
 }
 
 
-TEST(MetricsTest, Gauge)
+TEST_F(MetricsTest, Gauge)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
@@ -130,7 +167,7 @@ TEST(MetricsTest, Gauge)
 }
 
 
-TEST(MetricsTest, Statistics)
+TEST_F(MetricsTest, Statistics)
 {
   Counter counter("test/counter", process::TIME_SERIES_WINDOW);
 
@@ -164,7 +201,7 @@ TEST(MetricsTest, Statistics)
 }
 
 
-TEST(MetricsTest, Snapshot)
+TEST_F(MetricsTest, Snapshot)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
@@ -236,7 +273,7 @@ TEST(MetricsTest, Snapshot)
 }
 
 
-TEST(MetricsTest, SnapshotTimeout)
+TEST_F(MetricsTest, SnapshotTimeout)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
@@ -339,7 +376,7 @@ TEST(MetricsTest, SnapshotTimeout)
 
 
 // Ensures that the aggregate statistics are correct in the snapshot.
-TEST(MetricsTest, SnapshotStatistics)
+TEST_F(MetricsTest, SnapshotStatistics)
 {
   UPID upid("metrics", process::address());
 
@@ -404,7 +441,7 @@ TEST(MetricsTest, SnapshotStatistics)
 }
 
 
-TEST(MetricsTest, Timer)
+TEST_F(MetricsTest, Timer)
 {
   metrics::Timer<Nanoseconds> timer("test/timer");
   EXPECT_EQ("test/timer_ns", timer.name());
@@ -438,7 +475,7 @@ static Future<int> advanceAndReturn()
 }
 
 
-TEST(MetricsTest, AsyncTimer)
+TEST_F(MetricsTest, AsyncTimer)
 {
   metrics::Timer<Microseconds> t("test/timer");
   EXPECT_EQ("test/timer_us", t.name());
@@ -462,3 +499,45 @@ TEST(MetricsTest, AsyncTimer)
 
   AWAIT_READY(metrics::remove(t));
 }
+
+
+// Tests that the `/metrics/snapshot` endpoint rejects unauthenticated requests
+// when HTTP authentication is enabled.
+TEST_F(MetricsTest, SnapshotAuthenticationEnabled)
+{
+  ASSERT_TRUE(GTEST_IS_THREADSAFE);
+
+  process::Owned<Authenticator> authenticator(
+    new BasicAuthenticator(
+        DEFAULT_HTTP_AUTHENTICATION_REALM, {{"foo", "bar"}}));
+
+  AWAIT_READY(
+      setAuthenticator(DEFAULT_HTTP_AUTHENTICATION_REALM, authenticator));
+
+  UPID upid("metrics", process::address());
+
+  Clock::pause();
+
+  // Add a gauge and a counter.
+  GaugeProcess process;
+  PID<GaugeProcess> pid = spawn(&process);
+  ASSERT_TRUE(pid);
+
+  Gauge gauge("test/gauge", defer(pid, &GaugeProcess::get));
+  Gauge gaugeFail("test/gauge_fail", defer(pid, &GaugeProcess::fail));
+  Counter counter("test/counter");
+
+  AWAIT_READY(metrics::add(gauge));
+  AWAIT_READY(metrics::add(gaugeFail));
+  AWAIT_READY(metrics::add(counter));
+
+  // Advance the clock to avoid rate limit.
+  Clock::advance(Seconds(1));
+
+  // A request with no authentication header.
+  Future<Response> response = http::get(upid, "snapshot");
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Unauthorized({}).status, response);
+
+  terminate(process);
+  wait(process);
+}


[04/13] mesos git commit: Added authentication to `/metrics/snapshot` endpoint.

Posted by ka...@apache.org.
Added authentication to `/metrics/snapshot` endpoint.

This patch adds authentication to the `/metrics/snapshot`
endpoint on the master and agent.

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


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

Branch: refs/heads/master
Commit: 066fc4bd0df6690a5e1a929d3836e307c1e22586
Parents: 7b87dd5
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:02 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:43:02 2016 -0400

----------------------------------------------------------------------
 .../include/process/metrics/metrics.hpp         | 19 ++++++++++++-----
 3rdparty/libprocess/src/metrics/metrics.cpp     | 22 ++++++++++++++++----
 3rdparty/libprocess/src/process.cpp             |  9 +++++---
 3 files changed, 38 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/066fc4bd/3rdparty/libprocess/include/process/metrics/metrics.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/metrics/metrics.hpp b/3rdparty/libprocess/include/process/metrics/metrics.hpp
index 7abfadf..25e8fca 100644
--- a/3rdparty/libprocess/include/process/metrics/metrics.hpp
+++ b/3rdparty/libprocess/include/process/metrics/metrics.hpp
@@ -31,7 +31,7 @@ namespace process {
 namespace metrics {
 
 // Initializes the metrics library.
-void initialize();
+void initialize(const Option<std::string>& authenticationRealm = None());
 
 namespace internal {
 
@@ -50,16 +50,21 @@ protected:
 private:
   static std::string help();
 
-  MetricsProcess(const Option<Owned<RateLimiter>>& _limiter)
+  MetricsProcess(
+      const Option<Owned<RateLimiter>>& _limiter,
+      const Option<std::string>& _authenticationRealm)
     : ProcessBase("metrics"),
-      limiter(_limiter)
+      limiter(_limiter),
+      authenticationRealm(_authenticationRealm)
   {}
 
   // Non-copyable, non-assignable.
   MetricsProcess(const MetricsProcess&);
   MetricsProcess& operator=(const MetricsProcess&);
 
-  Future<http::Response> snapshot(const http::Request& request);
+  Future<http::Response> snapshot(
+      const http::Request& request,
+      const Option<std::string>& /* principal */);
   Future<http::Response> _snapshot(const http::Request& request);
   static std::list<Future<double> > _snapshotTimeout(
       const std::list<Future<double> >& futures);
@@ -76,7 +81,11 @@ private:
   Option<Owned<RateLimiter>> limiter;
 
   // Needed for access to the private constructor.
-  friend void process::metrics::initialize();
+  friend void process::metrics::initialize(
+      const Option<std::string>& authenticationRealm);
+
+  // The authentication realm that metrics HTTP endpoints are installed into.
+  const Option<std::string> authenticationRealm;
 };
 
 }  // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/066fc4bd/3rdparty/libprocess/src/metrics/metrics.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/metrics/metrics.cpp b/3rdparty/libprocess/src/metrics/metrics.cpp
index 7677dff..65b5843 100644
--- a/3rdparty/libprocess/src/metrics/metrics.cpp
+++ b/3rdparty/libprocess/src/metrics/metrics.cpp
@@ -44,7 +44,7 @@ namespace metrics {
 static internal::MetricsProcess* metrics_process = NULL;
 
 
-void initialize()
+void initialize(const Option<string>& authenticationRealm)
 {
   // To prevent a deadlock, we must ensure libprocess is
   // initialized. Otherwise, libprocess will be implicitly
@@ -101,7 +101,8 @@ void initialize()
       }
     }
 
-    metrics_process = new internal::MetricsProcess(limiter);
+    metrics_process =
+      new internal::MetricsProcess(limiter, authenticationRealm);
     spawn(metrics_process);
 
     initialized->done();
@@ -122,7 +123,18 @@ MetricsProcess* MetricsProcess::instance()
 
 void MetricsProcess::initialize()
 {
-  route("/snapshot", help(), &MetricsProcess::snapshot);
+  if (authenticationRealm.isSome()) {
+    route("/snapshot",
+          authenticationRealm.get(),
+          help(),
+          &MetricsProcess::snapshot);
+  } else {
+    route("/snapshot",
+          help(),
+          [this](const http::Request& request) {
+            return snapshot(request, None());
+          });
+  }
 }
 
 
@@ -165,7 +177,9 @@ Future<Nothing> MetricsProcess::remove(const string& name)
 }
 
 
-Future<http::Response> MetricsProcess::snapshot(const http::Request& request)
+Future<http::Response> MetricsProcess::snapshot(
+    const http::Request& request,
+    const Option<string>& /* principal */)
 {
   Future<Nothing> acquire = Nothing();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/066fc4bd/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 327ee43..ef27b66 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -966,6 +966,12 @@ bool initialize(
   // Create global garbage collector process.
   gc = spawn(new GarbageCollector());
 
+  // Initialize the metrics process. We need to initialize this before the other
+  // global processes because `metrics::initialize` is also called when metrics
+  // are added in other initialization code, and we want this to be the first
+  // initialization in order to populate the authentication realm correctly.
+  metrics::initialize(authenticationRealm);
+
   // Create global help process.
   help = spawn(new Help(delegate), true);
 
@@ -981,9 +987,6 @@ bool initialize(
   // Create the global HTTP authentication router.
   authenticator_manager = new AuthenticatorManager();
 
-  // Initialize the metrics process.
-  metrics::initialize();
-
   // Initialize the mime types.
   mime::initialize();
 


[10/13] mesos git commit: Updated gperftools to version 2.5 (libprocess).

Posted by ka...@apache.org.
Updated gperftools to version 2.5 (libprocess).

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


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

Branch: refs/heads/master
Commit: 5aec0377bdaba4aa72debdf37aeb66960e5192d1
Parents: 50a22f3
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:27 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:44:23 2016 -0400

----------------------------------------------------------------------
 .../libprocess/3rdparty/gperftools-2.0.tar.gz    | Bin 1261243 -> 0 bytes
 .../libprocess/3rdparty/gperftools-2.5.tar.gz    | Bin 0 -> 5216569 bytes
 3rdparty/libprocess/3rdparty/versions.am         |   2 +-
 3 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5aec0377/3rdparty/libprocess/3rdparty/gperftools-2.0.tar.gz
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/gperftools-2.0.tar.gz b/3rdparty/libprocess/3rdparty/gperftools-2.0.tar.gz
deleted file mode 100644
index 13b03ca..0000000
Binary files a/3rdparty/libprocess/3rdparty/gperftools-2.0.tar.gz and /dev/null differ

http://git-wip-us.apache.org/repos/asf/mesos/blob/5aec0377/3rdparty/libprocess/3rdparty/gperftools-2.5.tar.gz
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/gperftools-2.5.tar.gz b/3rdparty/libprocess/3rdparty/gperftools-2.5.tar.gz
new file mode 100644
index 0000000..0002c83
Binary files /dev/null and b/3rdparty/libprocess/3rdparty/gperftools-2.5.tar.gz differ

http://git-wip-us.apache.org/repos/asf/mesos/blob/5aec0377/3rdparty/libprocess/3rdparty/versions.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/versions.am b/3rdparty/libprocess/3rdparty/versions.am
index 5506eb3..4699543 100644
--- a/3rdparty/libprocess/3rdparty/versions.am
+++ b/3rdparty/libprocess/3rdparty/versions.am
@@ -5,7 +5,7 @@
 BOOST_VERSION = 1.53.0
 GLOG_VERSION = 0.3.3
 GMOCK_VERSION = 1.7.0
-GPERFTOOLS_VERSION = 2.0
+GPERFTOOLS_VERSION = 2.5
 LIBEV_VERSION = 4.22
 PROTOBUF_VERSION = 2.6.1
 HTTP_PARSER_VERSION = 2.6.2


[08/13] mesos git commit: Fixed the build when perftools is enabled.

Posted by ka...@apache.org.
Fixed the build when perftools is enabled.

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


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

Branch: refs/heads/master
Commit: 50a22f3d688faec95937090d3e6d75ae641dc940
Parents: 8ff034d
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:33 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:44:23 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/configure.ac     | 5 +++--
 3rdparty/libprocess/src/profiler.cpp | 8 ++++----
 2 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/50a22f3d/3rdparty/libprocess/configure.ac
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/configure.ac b/3rdparty/libprocess/configure.ac
index d27e46e..9bdd2f6 100644
--- a/3rdparty/libprocess/configure.ac
+++ b/3rdparty/libprocess/configure.ac
@@ -76,7 +76,7 @@ AC_ARG_ENABLE([optimize],
 AC_ARG_ENABLE([perftools],
               AS_HELP_STRING([--enable-perftools],
                              [enable google perftools]),
-              [gperftools=yes])
+              [gperftools=yes], [])
 
 AC_ARG_ENABLE([bundled],
               AS_HELP_STRING([--disable-bundled],
@@ -849,7 +849,8 @@ CONFIGURE_ARGS="$CONFIGURE_ARGS CXXFLAGS='$CXXFLAGS'"
 AX_PTHREAD([], [AC_MSG_ERROR([failed to find pthreads])])
 
 
-AM_CONDITIONAL([HAS_GPERFTOOLS], [test "x$gperftools" = "xyes"])
+AM_CONDITIONAL([HAS_GPERFTOOLS], [test x"$gperftools" = "xyes"])
+AM_COND_IF([HAS_GPERFTOOLS], AC_DEFINE([ENABLE_GPERFTOOLS], [1]))
 
 
 # Check if we should/can build with libz.

http://git-wip-us.apache.org/repos/asf/mesos/blob/50a22f3d/3rdparty/libprocess/src/profiler.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/profiler.cpp b/3rdparty/libprocess/src/profiler.cpp
index 58cf2fe..397d7b9 100644
--- a/3rdparty/libprocess/src/profiler.cpp
+++ b/3rdparty/libprocess/src/profiler.cpp
@@ -14,7 +14,7 @@
 
 #include <glog/logging.h>
 
-#ifdef HAS_GPERFTOOLS
+#ifdef ENABLE_GPERFTOOLS
 #include <gperftools/profiler.h>
 #endif
 
@@ -58,7 +58,7 @@ const std::string Profiler::STOP_HELP()
 
 Future<http::Response> Profiler::start(const http::Request& request)
 {
-#ifdef HAS_GPERFTOOLS
+#ifdef ENABLE_GPERFTOOLS
   const Option<std::string>
     enableProfiler = os::getenv("LIBPROCESS_ENABLE_PROFILER");
   if (enableProfiler.isNone() || enableProfiler.get() != "1") {
@@ -82,7 +82,7 @@ Future<http::Response> Profiler::start(const http::Request& request)
   // here:
   // https://groups.google.com/d/topic/google-perftools/Df10Uy4Djrg/discussion
   // NOTE: We have not tested this with libunwind > 1.0.1.
-  if (!ProfilerStart(PROFILE_FILE.c_str())) {
+  if (!ProfilerStart(PROFILE_FILE)) {
     Try<std::string> error =
       strings::format("Failed to start profiler: %s", os::strerror(errno));
     LOG(ERROR) << error.get();
@@ -101,7 +101,7 @@ Future<http::Response> Profiler::start(const http::Request& request)
 
 Future<http::Response> Profiler::stop(const http::Request& request)
 {
-#ifdef HAS_GPERFTOOLS
+#ifdef ENABLE_GPERFTOOLS
   if (!started) {
     return http::BadRequest("Profiler not running.\n");
   }


[03/13] mesos git commit: Added authentication to `/logging/toggle` endpoint.

Posted by ka...@apache.org.
Added authentication to `/logging/toggle` endpoint.

This patch adds authentication to the `/logging/toggle`
HTTP endpoint on the master and agent.

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


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

Branch: refs/heads/master
Commit: 7b87dd5861e3df94be41aa0210a1e81eab8459d0
Parents: 23bff54
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:42:48 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:42:48 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/include/process/logging.hpp | 23 ++++++++++++++++----
 3rdparty/libprocess/src/logging.cpp             |  4 +++-
 3rdparty/libprocess/src/process.cpp             |  2 +-
 3 files changed, 23 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7b87dd58/3rdparty/libprocess/include/process/logging.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/logging.hpp b/3rdparty/libprocess/include/process/logging.hpp
index 0c07746..651cb37 100644
--- a/3rdparty/libprocess/include/process/logging.hpp
+++ b/3rdparty/libprocess/include/process/logging.hpp
@@ -25,9 +25,10 @@ namespace process {
 class Logging : public Process<Logging>
 {
 public:
-  Logging()
+  Logging(Option<std::string> _authenticationRealm)
     : ProcessBase("logging"),
-      original(FLAGS_v)
+      original(FLAGS_v),
+      authenticationRealm(_authenticationRealm)
   {
     // Make sure all reads/writes can be done atomically (i.e., to
     // make sure VLOG(*) statements don't read partial writes).
@@ -41,11 +42,21 @@ public:
 protected:
   virtual void initialize()
   {
-    route("/toggle", TOGGLE_HELP(), &This::toggle);
+    if (authenticationRealm.isSome()) {
+      route("/toggle", authenticationRealm.get(), TOGGLE_HELP(), &This::toggle);
+    } else {
+      route("/toggle",
+            TOGGLE_HELP(),
+            [this](const http::Request& request) {
+              return This::toggle(request, None());
+            });
+    }
   }
 
 private:
-  Future<http::Response> toggle(const http::Request& request);
+  Future<http::Response> toggle(
+      const http::Request& request,
+      const Option<std::string>& /* principal */);
 
   void set(int v)
   {
@@ -74,6 +85,10 @@ private:
   Timeout timeout;
 
   const int32_t original; // Original value of FLAGS_v.
+
+  // The authentication realm that the `/logging/toggle` endpoint will be
+  // installed into.
+  const Option<std::string> authenticationRealm;
 };
 
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b87dd58/3rdparty/libprocess/src/logging.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/logging.cpp b/3rdparty/libprocess/src/logging.cpp
index bdbf716..1cb0f4a 100644
--- a/3rdparty/libprocess/src/logging.cpp
+++ b/3rdparty/libprocess/src/logging.cpp
@@ -25,7 +25,9 @@
 
 namespace process {
 
-Future<http::Response> Logging::toggle(const http::Request& request)
+Future<http::Response> Logging::toggle(
+    const http::Request& request,
+    const Option<std::string>& /* principal */)
 {
   Option<std::string> level = request.url.query.get("level");
   Option<std::string> duration = request.url.query.get("duration");

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b87dd58/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index d1d924c..327ee43 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -970,7 +970,7 @@ bool initialize(
   help = spawn(new Help(delegate), true);
 
   // Create the global logging process.
-  spawn(new Logging(), true);
+  spawn(new Logging(authenticationRealm), true);
 
   // Create the global profiler process.
   spawn(new Profiler(), true);


[09/13] mesos git commit: Added authentication to the '/profiler/*' endpoints.

Posted by ka...@apache.org.
Added authentication to the '/profiler/*' endpoints.

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


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

Branch: refs/heads/master
Commit: 36aeb17e0bf41e630e6020af9832dfb853817b68
Parents: 0956ce8
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:42 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:44:23 2016 -0400

----------------------------------------------------------------------
 .../libprocess/include/process/profiler.hpp     | 42 +++++++++++++++++---
 3rdparty/libprocess/src/process.cpp             |  2 +-
 3rdparty/libprocess/src/profiler.cpp            |  8 +++-
 3 files changed, 44 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/36aeb17e/3rdparty/libprocess/include/process/profiler.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/profiler.hpp b/3rdparty/libprocess/include/process/profiler.hpp
index b305f11..f6fccfb 100644
--- a/3rdparty/libprocess/include/process/profiler.hpp
+++ b/3rdparty/libprocess/include/process/profiler.hpp
@@ -24,15 +24,39 @@ namespace process {
 class Profiler : public Process<Profiler>
 {
 public:
-  Profiler() : ProcessBase("profiler"), started(false) {}
+  Profiler(const Option<std::string>& _authenticationRealm)
+    : ProcessBase("profiler"),
+      started(false),
+      authenticationRealm(_authenticationRealm) {}
 
   virtual ~Profiler() {}
 
 protected:
   virtual void initialize()
   {
-    route("/start", START_HELP(), &Profiler::start);
-    route("/stop", STOP_HELP(), &Profiler::stop);
+    if (authenticationRealm.isSome()) {
+      route("/start",
+            authenticationRealm.get(),
+            START_HELP(),
+            &Profiler::start);
+
+      route("/stop",
+            authenticationRealm.get(),
+            STOP_HELP(),
+            &Profiler::stop);
+    } else {
+      route("/start",
+            START_HELP(),
+            [this](const http::Request& request) {
+              return Profiler::start(request, None());
+            });
+
+      route("/stop",
+            STOP_HELP(),
+            [this](const http::Request& request) {
+              return Profiler::stop(request, None());
+            });
+    }
   }
 
 private:
@@ -42,14 +66,22 @@ private:
   // HTTP endpoints.
 
   // Starts the profiler. There are no request parameters.
-  Future<http::Response> start(const http::Request& request);
+  Future<http::Response> start(
+      const http::Request& request,
+      const Option<std::string>& /* principal */);
 
   // Stops the profiler. There are no request parameters.
   // This returns the profile output, it will also remain present
   // in the working directory.
-  Future<http::Response> stop(const http::Request& request);
+  Future<http::Response> stop(
+      const http::Request& request,
+      const Option<std::string>& /* principal */);
 
   bool started;
+
+  // The authentication realm that the profiler's HTTP endpoints will be
+  // installed into.
+  Option<std::string> authenticationRealm;
 };
 
 } // namespace process {

http://git-wip-us.apache.org/repos/asf/mesos/blob/36aeb17e/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index ef27b66..dcaa646 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -979,7 +979,7 @@ bool initialize(
   spawn(new Logging(authenticationRealm), true);
 
   // Create the global profiler process.
-  spawn(new Profiler(), true);
+  spawn(new Profiler(authenticationRealm), true);
 
   // Create the global system statistics process.
   spawn(new System(), true);

http://git-wip-us.apache.org/repos/asf/mesos/blob/36aeb17e/3rdparty/libprocess/src/profiler.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/profiler.cpp b/3rdparty/libprocess/src/profiler.cpp
index 397d7b9..9b2f106 100644
--- a/3rdparty/libprocess/src/profiler.cpp
+++ b/3rdparty/libprocess/src/profiler.cpp
@@ -56,7 +56,9 @@ const std::string Profiler::STOP_HELP()
 }
 
 
-Future<http::Response> Profiler::start(const http::Request& request)
+Future<http::Response> Profiler::start(
+    const http::Request& request,
+    const Option<std::string>& /* principal */)
 {
 #ifdef ENABLE_GPERFTOOLS
   const Option<std::string>
@@ -99,7 +101,9 @@ Future<http::Response> Profiler::start(const http::Request& request)
 }
 
 
-Future<http::Response> Profiler::stop(const http::Request& request)
+Future<http::Response> Profiler::stop(
+    const http::Request& request,
+    const Option<std::string>& /* principal */)
 {
 #ifdef ENABLE_GPERFTOOLS
   if (!started) {


[13/13] mesos git commit: Added the '--enable-perftools' flag to configure docs.

Posted by ka...@apache.org.
Added the '--enable-perftools' flag to configure docs.

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


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

Branch: refs/heads/master
Commit: dc8b9a4abff74d1a2e33039836682bdfa7188f87
Parents: 4bc17a1
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:50 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:47:07 2016 -0400

----------------------------------------------------------------------
 docs/configuration.md | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/dc8b9a4a/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 318275f..2796a81 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -2004,6 +2004,14 @@ quotas for container sandbox directories. Valid project IDs range from
   </tr>
   <tr>
     <td>
+      --enable-perftools
+    </td>
+    <td>
+      Whether profiling with Google perftools is enabled. [default=no]
+    </td>
+  </tr>
+  <tr>
+    <td>
       --disable-bundled
     </td>
     <td>


[06/13] mesos git commit: Added new MetricsTests with authentication to Mesos tests.

Posted by ka...@apache.org.
Added new MetricsTests with authentication to Mesos tests.

The tests `MetricsTest.AgentAuthenticationEnabled` and
`MetricsTest.MasterAuthenticationEnabled` are added in
this patch.

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


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

Branch: refs/heads/master
Commit: 6a6b999d4ee19db3de65a04557f4673041a4f826
Parents: e0aaccf
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:14 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:43:14 2016 -0400

----------------------------------------------------------------------
 src/tests/metrics_tests.cpp | 99 +++++++++++++++++++++++++++++++++++++++-
 1 file changed, 98 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6a6b999d/src/tests/metrics_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/metrics_tests.cpp b/src/tests/metrics_tests.cpp
index eacff67..2ff6bb5 100644
--- a/src/tests/metrics_tests.cpp
+++ b/src/tests/metrics_tests.cpp
@@ -16,6 +16,8 @@
 
 #include <gtest/gtest.h>
 
+#include <mesos/authentication/http/basic_authenticator_factory.hpp>
+
 #include <process/future.hpp>
 #include <process/http.hpp>
 #include <process/owned.hpp>
@@ -28,6 +30,10 @@
 
 #include "tests/mesos.hpp"
 
+namespace authentication = process::http::authentication;
+
+using mesos::http::authentication::BasicAuthenticatorFactory;
+
 using mesos::internal::master::Master;
 using mesos::internal::slave::Slave;
 
@@ -39,7 +45,44 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-class MetricsTest : public mesos::internal::tests::MesosTest {};
+class MetricsTest : public mesos::internal::tests::MesosTest
+{
+protected:
+  void setBasicHttpAuthenticator(
+      const std::string& realm,
+      const Credentials& credentials)
+  {
+    Try<authentication::Authenticator*> authenticator =
+      BasicAuthenticatorFactory::create(realm, credentials);
+
+    ASSERT_SOME(authenticator);
+
+    // Add this realm to the set of realms which will be unset during teardown.
+    realms.insert(realm);
+
+    // Pass ownership of the authenticator to libprocess.
+    AWAIT_READY(authentication::setAuthenticator(
+        realm,
+        Owned<authentication::Authenticator>(authenticator.get())));
+  }
+
+  virtual void TearDown()
+  {
+    foreach (const std::string& realm, realms) {
+      // We need to wait in order to ensure that the operation completes before
+      // we leave `TearDown`. Otherwise, we may leak a mock object.
+      AWAIT_READY(authentication::unsetAuthenticator(realm));
+    }
+
+    realms.clear();
+
+    MesosTest::TearDown();
+  }
+
+private:
+  hashset<std::string> realms;
+};
+
 
 TEST_F(MetricsTest, Master)
 {
@@ -220,6 +263,60 @@ TEST_F(MetricsTest, Slave)
   EXPECT_EQ(1u, stats.values.count("slave/disk_percent"));
 }
 
+
+// Tests that the `/metrics/snapshot` endpoint will reject unauthenticated
+// requests when HTTP authentication is enabled on the master.
+TEST_F(MetricsTest, MasterAuthenticationEnabled)
+{
+  Credentials credentials;
+  credentials.add_credentials()->CopyFrom(DEFAULT_CREDENTIAL);
+
+  // Create a basic HTTP authenticator with the specified credentials and set it
+  // as the authenticator for `DEFAULT_HTTP_AUTHENTICATION_REALM`.
+  setBasicHttpAuthenticator(DEFAULT_HTTP_AUTHENTICATION_REALM, credentials);
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  // Get the snapshot.
+  process::UPID upid("metrics", process::address());
+
+  process::Future<process::http::Response> response =
+      process::http::get(upid, "snapshot");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      process::http::Unauthorized({}).status, response);
+}
+
+
+// Tests that the `/metrics/snapshot` endpoint will reject unauthenticated
+// requests when HTTP authentication is enabled on the agent.
+TEST_F(MetricsTest, AgentAuthenticationEnabled)
+{
+  Credentials credentials;
+  credentials.add_credentials()->CopyFrom(DEFAULT_CREDENTIAL);
+
+  // Create a basic HTTP authenticator with the specified credentials and set it
+  // as the authenticator for `DEFAULT_HTTP_AUTHENTICATION_REALM`.
+  setBasicHttpAuthenticator(DEFAULT_HTTP_AUTHENTICATION_REALM, credentials);
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+  Try<Owned<cluster::Slave>> agent = StartSlave(detector.get());
+  ASSERT_SOME(agent);
+
+  // Get the snapshot.
+  process::UPID upid("metrics", process::address());
+
+  process::Future<process::http::Response> response =
+      process::http::get(upid, "snapshot");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      process::http::Unauthorized({}).status, response);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[02/13] mesos git commit: Added a realm parameter to 'process::initialize' (Mesos).

Posted by ka...@apache.org.
Added a realm parameter to 'process::initialize' (Mesos).

In order to enable authentication on libprocess-level
HTTP endpoints, this patch adds code to the master and
agent's main.cpp file which makes use of the new
`authenticationRealm` argument to `process::initialize`
which allows the authentication realm of such endpoints
to be set when libprocess is initialized. The argument is
added to libprocess initialization in the tests as well.

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


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

Branch: refs/heads/master
Commit: 23bff549128c50586d34cda110e6f02021242973
Parents: 3d78119
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:42:42 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:42:42 2016 -0400

----------------------------------------------------------------------
 src/master/main.cpp | 10 ++++++++--
 src/slave/main.cpp  |  8 +++++++-
 src/tests/main.cpp  | 10 ++++++++--
 src/tests/mesos.hpp |  2 ++
 4 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/23bff549/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index e1b6d48..23149d5 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -240,8 +240,14 @@ int main(int argc, char** argv)
     }
   }
 
-  // Initialize libprocess.
-  process::initialize("master");
+  // This should be the first invocation of `process::initialize`. If it returns
+  // `false`, then it has already been called, which means that the
+  // authentication realm for libprocess-level HTTP endpoints was not set to the
+  // correct value for the master.
+  if (!process::initialize("master", DEFAULT_HTTP_AUTHENTICATION_REALM)) {
+    EXIT(EXIT_FAILURE) << "The call to `process::initialize()` in the master's "
+                       << "`main()` was not the function's first invocation";
+  }
 
   logging::initialize(argv[0], flags, true); // Catch signals.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/23bff549/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index 3ea1453..fee46ba 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -211,7 +211,13 @@ int main(int argc, char** argv)
 
   const string id = process::ID::generate("slave"); // Process ID.
 
-  process::initialize(id);
+  // If `process::initialize()` returns `false`, then it was called before this
+  // invocation, meaning the authentication realm for libprocess-level HTTP
+  // endpoints was set incorrectly. This should be the first invocation.
+  if (!process::initialize(id, DEFAULT_HTTP_AUTHENTICATION_REALM)) {
+    EXIT(EXIT_FAILURE) << "The call to `process::initialize()` in the agent's "
+                       << "`main()` was not the function's first invocation";
+  }
 
   logging::initialize(argv[0], flags, true); // Catch signals.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/23bff549/src/tests/main.cpp
----------------------------------------------------------------------
diff --git a/src/tests/main.cpp b/src/tests/main.cpp
index 1425850..c3ccf91 100644
--- a/src/tests/main.cpp
+++ b/src/tests/main.cpp
@@ -31,6 +31,7 @@
 
 #include "tests/environment.hpp"
 #include "tests/flags.hpp"
+#include "tests/mesos.hpp"
 #include "tests/module.hpp"
 
 using namespace mesos::internal;
@@ -74,8 +75,13 @@ int main(int argc, char** argv)
   // overwrite whatever the user set.
   os::setenv("LIBPROCESS_METRICS_SNAPSHOT_ENDPOINT_RATE_LIMIT", "", false);
 
-  // Initialize libprocess.
-  process::initialize();
+  // If `process::initialize()` returns `false`, then it was called before this
+  // invocation, meaning the authentication realm for libprocess-level HTTP
+  // endpoints was set incorrectly. This should be the first invocation.
+  if (!process::initialize(None(), DEFAULT_HTTP_AUTHENTICATION_REALM)) {
+    EXIT(EXIT_FAILURE) << "The call to `process::initialize()` in the tests' "
+                       << "`main()` was not the function's first invocation";
+  }
 
   // Be quiet by default!
   if (!flags.verbose) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/23bff549/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index bff7938..78edab8 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -98,6 +98,8 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
+constexpr char DEFAULT_HTTP_AUTHENTICATION_REALM[] = "test-realm";
+
 // Forward declarations.
 class MockExecutor;
 


[07/13] mesos git commit: Added a LoggingTest with authentication.

Posted by ka...@apache.org.
Added a LoggingTest with authentication.

The test `LoggingTest.ToggleAuthenticationEnabled`
is added in this patch.

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


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

Branch: refs/heads/master
Commit: 8ff034d3e85cd3109d76212ad5a04ac260526205
Parents: 6a6b999
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:18 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:43:18 2016 -0400

----------------------------------------------------------------------
 src/tests/logging_tests.cpp | 70 +++++++++++++++++++++++++++++++++++++++-
 1 file changed, 69 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8ff034d3/src/tests/logging_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/logging_tests.cpp b/src/tests/logging_tests.cpp
index 675a3f8..5412744 100644
--- a/src/tests/logging_tests.cpp
+++ b/src/tests/logging_tests.cpp
@@ -16,6 +16,8 @@
 
 #include <gmock/gmock.h>
 
+#include <mesos/authentication/http/basic_authenticator_factory.hpp>
+
 #include <process/future.hpp>
 #include <process/gtest.hpp>
 #include <process/http.hpp>
@@ -24,16 +26,61 @@
 
 #include "logging/logging.hpp"
 
+#include "tests/mesos.hpp"
+
+namespace authentication = process::http::authentication;
+
+using mesos::http::authentication::BasicAuthenticatorFactory;
+
 using process::http::BadRequest;
 using process::http::OK;
 using process::http::Response;
+using process::http::Unauthorized;
 
 namespace mesos {
 namespace internal {
 namespace tests {
 
+class LoggingTest : public mesos::internal::tests::MesosTest
+{
+protected:
+  void setBasicHttpAuthenticator(
+      const std::string& realm,
+      const Credentials& credentials)
+  {
+    Try<authentication::Authenticator*> authenticator =
+      BasicAuthenticatorFactory::create(realm, credentials);
+
+    ASSERT_SOME(authenticator);
+
+    // Add this realm to the set of realms which will be unset during teardown.
+    realms.insert(realm);
+
+    // Pass ownership of the authenticator to libprocess.
+    AWAIT_READY(authentication::setAuthenticator(
+        realm,
+        process::Owned<authentication::Authenticator>(authenticator.get())));
+  }
+
+  virtual void TearDown()
+  {
+    foreach (const std::string& realm, realms) {
+      // We need to wait in order to ensure that the operation completes before
+      // we leave `TearDown`. Otherwise, we may leak a mock object.
+      AWAIT_READY(authentication::unsetAuthenticator(realm));
+    }
+
+    realms.clear();
+
+    MesosTest::TearDown();
+  }
+
+private:
+  hashset<std::string> realms;
+};
+
 
-TEST(LoggingTest, Toggle)
+TEST_F(LoggingTest, Toggle)
 {
   process::PID<> pid;
   pid.id = "logging";
@@ -79,6 +126,27 @@ TEST(LoggingTest, Toggle)
       response);
 }
 
+
+// Tests that the `/logging/toggle` endpoint rejects unauthenticated requests
+// when HTTP authentication is enabled.
+TEST_F(LoggingTest, ToggleAuthenticationEnabled)
+{
+  Credentials credentials;
+  credentials.add_credentials()->CopyFrom(DEFAULT_CREDENTIAL);
+
+  // Create a basic HTTP authenticator with the specified credentials and set it
+  // as the authenticator for `DEFAULT_HTTP_AUTHENTICATION_REALM`.
+  setBasicHttpAuthenticator(DEFAULT_HTTP_AUTHENTICATION_REALM, credentials);
+
+  process::PID<> pid;
+  pid.id = "logging";
+  pid.address = process::address();
+
+  process::Future<Response> response = process::http::get(pid, "toggle");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Unauthorized({}).status, response);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[12/13] mesos git commit: Added tests for the Profiler to libprocess.

Posted by ka...@apache.org.
Added tests for the Profiler to libprocess.

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


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

Branch: refs/heads/master
Commit: 4bc17a106f2d40e5c41bb0d0dedf9544952ee2e4
Parents: 36aeb17
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Apr 26 10:43:46 2016 -0400
Committer: Kapil Arya <ka...@mesosphere.io>
Committed: Tue Apr 26 10:47:01 2016 -0400

----------------------------------------------------------------------
 3rdparty/libprocess/Makefile.am                 |   1 +
 .../libprocess/src/tests/profiler_tests.cpp     | 136 +++++++++++++++++++
 2 files changed, 137 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4bc17a10/3rdparty/libprocess/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/Makefile.am b/3rdparty/libprocess/Makefile.am
index c51c31e..51f62aa 100644
--- a/3rdparty/libprocess/Makefile.am
+++ b/3rdparty/libprocess/Makefile.am
@@ -158,6 +158,7 @@ libprocess_tests_SOURCES =					\
   src/tests/mutex_tests.cpp					\
   src/tests/owned_tests.cpp					\
   src/tests/process_tests.cpp					\
+  src/tests/profiler_tests.cpp					\
   src/tests/queue_tests.cpp					\
   src/tests/reap_tests.cpp					\
   src/tests/sequence_tests.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/4bc17a10/3rdparty/libprocess/src/tests/profiler_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/profiler_tests.cpp b/3rdparty/libprocess/src/tests/profiler_tests.cpp
new file mode 100644
index 0000000..bf7a375
--- /dev/null
+++ b/3rdparty/libprocess/src/tests/profiler_tests.cpp
@@ -0,0 +1,136 @@
+// Licensed 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 <gtest/gtest.h>
+
+#include <string>
+
+#include <stout/duration.hpp>
+#include <stout/gtest.hpp>
+
+#include <process/authenticator.hpp>
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+#include <process/http.hpp>
+#include <process/process.hpp>
+#include <process/profiler.hpp>
+
+namespace authentication = process::http::authentication;
+namespace http = process::http;
+
+using authentication::Authenticator;
+using authentication::BasicAuthenticator;
+
+using http::BadRequest;
+using http::OK;
+using http::Response;
+using http::Unauthorized;
+
+using process::DEFAULT_HTTP_AUTHENTICATION_REALM;
+using process::Future;
+using process::UPID;
+
+using std::string;
+
+
+// TODO(greggomann): Move this into a base class in 'mesos.hpp'.
+class ProfilerTest : public ::testing::Test
+{
+protected:
+  Future<Nothing> setAuthenticator(
+      const string& realm,
+      process::Owned<Authenticator> authenticator)
+  {
+    realms.insert(realm);
+
+    return authentication::setAuthenticator(realm, authenticator);
+  }
+
+  virtual void TearDown()
+  {
+    foreach (const string& realm, realms) {
+      // We need to wait in order to ensure that the operation
+      // completes before we leave TearDown. Otherwise, we may
+      // leak a mock object.
+      AWAIT_READY(authentication::unsetAuthenticator(realm));
+    }
+    realms.clear();
+  }
+
+private:
+  hashset<string> realms;
+};
+
+
+// Tests that the profiler's HTTP endpoints return the correct responses
+// based on whether or not the profiler has been enabled.
+TEST_F(ProfilerTest, StartAndStop)
+{
+  UPID upid("profiler", process::address());
+
+  Future<Response> response = http::get(upid, "start");
+#ifdef ENABLE_GPERFTOOLS
+  Option<string> profilerEnabled = os::getenv("LIBPROCESS_ENABLE_PROFILER");
+
+  if (profilerEnabled.isSome() && profilerEnabled.get() == "1") {
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+    AWAIT_EXPECT_RESPONSE_BODY_EQ("Profiler started.\n", response);
+
+    response = http::get(upid, "stop");
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+  } else {
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+    AWAIT_EXPECT_RESPONSE_BODY_EQ(
+        "The profiler is not enabled. To enable the profiler, libprocess must "
+        "be started with LIBPROCESS_ENABLE_PROFILER=1 in the environment.\n",
+        response);
+
+    response = http::get(upid, "stop");
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+    AWAIT_EXPECT_RESPONSE_BODY_EQ("Profiler not running.\n", response);
+  }
+#else
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+  AWAIT_EXPECT_RESPONSE_BODY_EQ(
+      "Perftools is disabled. To enable perftools, "
+      "configure libprocess with --enable-perftools.\n",
+      response);
+
+  response = http::get(upid, "stop");
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+  AWAIT_EXPECT_RESPONSE_BODY_EQ(
+      "Perftools is disabled. To enable perftools, "
+      "configure libprocess with --enable-perftools.\n",
+      response);
+#endif
+}
+
+
+// Tests that the profiler's HTTP endpoints reject unauthenticated
+// requests when HTTP authentication is enabled.
+TEST_F(ProfilerTest, StartAndStopAuthenticationEnabled)
+{
+  process::Owned<Authenticator> authenticator(
+    new BasicAuthenticator(
+        DEFAULT_HTTP_AUTHENTICATION_REALM, {{"foo", "bar"}}));
+
+  AWAIT_READY(
+      setAuthenticator(DEFAULT_HTTP_AUTHENTICATION_REALM, authenticator));
+
+  UPID upid("profiler", process::address());
+
+  Future<Response> response = http::get(upid, "start");
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Unauthorized({}).status, response);
+
+  response = http::get(upid, "stop");
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Unauthorized({}).status, response);
+}