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

[1/2] mesos git commit: Added firewall mechanism to control access on libprocess http endpoints.

Repository: mesos
Updated Branches:
  refs/heads/master 18c4a3aaf -> 3ef08fafd


Added firewall mechanism to control access on libprocess http endpoints.

Introduces the interface `FirewallRule` which will be matched against
incoming connections in order to allow them to be served or being
blocked. For details, check the [design doc](https://goo.gl/f5ECQO).

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


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

Branch: refs/heads/master
Commit: 70c75c04d69d0f043186a7ec63f6e9702d8365b4
Parents: 18c4a3a
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Mon Jun 8 14:15:32 2015 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Mon Jun 8 14:15:33 2015 +0200

----------------------------------------------------------------------
 3rdparty/libprocess/include/Makefile.am         |   1 +
 .../libprocess/include/process/firewall.hpp     |  93 +++++++++++
 3rdparty/libprocess/include/process/process.hpp |  17 ++
 3rdparty/libprocess/src/process.cpp             |  53 +++++++
 3rdparty/libprocess/src/tests/process_tests.cpp | 156 +++++++++++++++++++
 5 files changed, 320 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/70c75c04/3rdparty/libprocess/include/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/Makefile.am b/3rdparty/libprocess/include/Makefile.am
index f45e7c5..d01880c 100644
--- a/3rdparty/libprocess/include/Makefile.am
+++ b/3rdparty/libprocess/include/Makefile.am
@@ -12,6 +12,7 @@ nobase_include_HEADERS =		\
   process/event.hpp			\
   process/executor.hpp			\
   process/filter.hpp			\
+  process/firewall.hpp			\
   process/future.hpp			\
   process/gc.hpp			\
   process/gmock.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/70c75c04/3rdparty/libprocess/include/process/firewall.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/firewall.hpp b/3rdparty/libprocess/include/process/firewall.hpp
new file mode 100644
index 0000000..16ed852
--- /dev/null
+++ b/3rdparty/libprocess/include/process/firewall.hpp
@@ -0,0 +1,93 @@
+/**
+ * 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
+ */
+
+#ifndef __PROCESS_FIREWALL_HPP__
+#define __PROCESS_FIREWALL_HPP__
+
+#include <string>
+
+#include <process/http.hpp>
+#include <process/socket.hpp>
+
+#include <stout/hashset.hpp>
+
+namespace process {
+namespace firewall {
+
+/**
+ * A 'FirewallRule' is an interface which allows control over which
+ * incoming HTTP connections are allowed. Concrete classes based on
+ * this interface must implement the 'apply' method; this method
+ * receives as parameters the socket where the connection is being
+ * initiated, as well as the request itself.
+ *
+ * Rules can be installed using the free function
+ * 'process::firewall::install()' defined in 'process.hpp'
+ */
+class FirewallRule
+{
+public:
+  FirewallRule() {}
+  virtual ~FirewallRule() {}
+
+  /**
+   * Method used to do inspection of incoming HTTP requests. It allows
+   * implementations to verify conditions on the requests and notify
+   * the caller if the rule has been broken.
+   *
+   * @param socket Socket used to attempt the HTTP connection.
+   * @param request HTTP request made by the client to libprocess.
+   * @return If the condition verification fails, i.e. the condition
+   *     has been broken, the returned Try contains an error.
+   */
+  virtual Try<Nothing> apply(
+      const network::Socket& socket,
+      const http::Request& request) = 0;
+};
+
+
+/**
+ * Simple firewall rule to reject any connection requesting a path
+ * in the provided list of disabled endpoints.
+ *
+ * Matches are required to be exact, no substrings nor wildcards are
+ * considered for a match.
+ */
+class DisabledEndpointsFirewallRule : public FirewallRule
+{
+public:
+  explicit DisabledEndpointsFirewallRule(const hashset<std::string>& _paths)
+    : paths(_paths) {}
+
+  virtual ~DisabledEndpointsFirewallRule() {}
+
+  virtual Try<Nothing> apply(
+      const network::Socket&,
+      const http::Request& request)
+  {
+    if (paths.contains(request.path)) {
+      return Error("'" + request.path + "' is disabled");
+    }
+
+    return Nothing();
+  }
+
+private:
+  hashset<std::string> paths;
+};
+
+} // namespace firewall {
+} // namespace process {
+
+#endif // __PROCESS_FIREWALL_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/70c75c04/3rdparty/libprocess/include/process/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/process.hpp b/3rdparty/libprocess/include/process/process.hpp
index 79d1719..e70dd38 100644
--- a/3rdparty/libprocess/include/process/process.hpp
+++ b/3rdparty/libprocess/include/process/process.hpp
@@ -11,6 +11,7 @@
 #include <process/clock.hpp>
 #include <process/event.hpp>
 #include <process/filter.hpp>
+#include <process/firewall.hpp>
 #include <process/http.hpp>
 #include <process/message.hpp>
 #include <process/mime.hpp>
@@ -23,6 +24,22 @@
 
 namespace process {
 
+namespace firewall {
+/**
+ * Installs the list of firewall rules to be used to allow or reject
+ * incoming connections. Each incoming connection will be tested
+ * against each rule in the order in which they appear in the vector
+ * 'rules'. As soon as any of the tests (calling the apply method)
+ * fails, the connection is rejected. If no test fails, the connection
+ * is allowed.
+ *
+ * @param rules List of rules which will be applied to incoming
+ * connections.
+ */
+void install(std::vector<Owned<FirewallRule>>&& rules);
+
+} // namespace firewall {
+
 class ProcessBase : public EventVisitor
 {
 public:

http://git-wip-us.apache.org/repos/asf/mesos/blob/70c75c04/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index d1b4d46..aadd7bb 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -40,6 +40,7 @@
 #include <sstream>
 #include <stack>
 #include <stdexcept>
+#include <utility>
 #include <vector>
 
 #include <process/address.hpp>
@@ -85,12 +86,14 @@
 #include "gate.hpp"
 #include "process_reference.hpp"
 
+using namespace process::firewall;
 using namespace process::metrics::internal;
 
 using process::wait; // Necessary on some OS's to disambiguate.
 
 using process::http::Accepted;
 using process::http::BadRequest;
+using process::http::Forbidden;
 using process::http::InternalServerError;
 using process::http::NotFound;
 using process::http::OK;
@@ -357,6 +360,8 @@ public:
   void terminate(const UPID& pid, bool inject, ProcessBase* sender = NULL);
   bool wait(const UPID& pid);
 
+  void installFirewall(std::vector<Owned<FirewallRule>>&& rules);
+
   void enqueue(ProcessBase* process);
   ProcessBase* dequeue();
 
@@ -382,6 +387,10 @@ private:
 
   // Number of running processes, to support Clock::settle operation.
   int running;
+
+  // List of rules applied to all incoming HTTP connections.
+  std::vector<Owned<FirewallRule>> firewallRules;
+  std::recursive_mutex firewall_mutex;
 };
 
 
@@ -674,6 +683,17 @@ void on_accept(const Future<Socket>& socket)
 } // namespace internal {
 
 
+namespace firewall {
+
+void install(std::vector<Owned<FirewallRule>>&& rules)
+{
+  process::initialize();
+
+  process_manager->installFirewall(std::move(rules));
+}
+
+} // namespace firewall {
+
 void initialize(const string& delegate)
 {
   // TODO(benh): Return an error if attempting to initialize again
@@ -1962,6 +1982,31 @@ bool ProcessManager::handle(
     return false;
   }
 
+  synchronized (firewall_mutex) {
+    foreach (Owned<FirewallRule>& rule, firewallRules) {
+      Try<Nothing> applied = rule->apply(socket, *request);
+      if (applied.isError()) {
+        VLOG(1) << "Returning '403 Forbidden' for '" << request->path
+                << "' (firewall rule forbids connection): "
+                << applied.error();
+        // TODO(arojas): Get rid of the duplicated code to return an
+        // error.
+
+        // Get the HttpProxy pid for this socket.
+        PID<HttpProxy> proxy = socket_manager->proxy(socket);
+
+        // Enqueue the response with the HttpProxy so that it respects
+        // the order of requests to account for HTTP/1.1 pipelining.
+        dispatch(
+            proxy, &HttpProxy::enqueue, Forbidden(applied.error()), *request);
+
+        // Cleanup request.
+        delete request;
+        return false;
+      }
+    }
+  }
+
   // Split the path by '/'.
   vector<string> tokens = strings::tokenize(request->path, "/");
 
@@ -2414,6 +2459,14 @@ bool ProcessManager::wait(const UPID& pid)
 }
 
 
+void ProcessManager::installFirewall(std::vector<Owned<FirewallRule>>&& rules)
+{
+  synchronized (firewall_mutex) {
+    firewallRules = std::move(rules);
+  }
+}
+
+
 void ProcessManager::enqueue(ProcessBase* process)
 {
   CHECK(process != NULL);

http://git-wip-us.apache.org/repos/asf/mesos/blob/70c75c04/3rdparty/libprocess/src/tests/process_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/process_tests.cpp b/3rdparty/libprocess/src/tests/process_tests.cpp
index 7b9ba9e..0832c62 100644
--- a/3rdparty/libprocess/src/tests/process_tests.cpp
+++ b/3rdparty/libprocess/src/tests/process_tests.cpp
@@ -30,6 +30,7 @@
 #include <stout/duration.hpp>
 #include <stout/gtest.hpp>
 #include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/nothing.hpp>
 #include <stout/os.hpp>
@@ -41,6 +42,9 @@
 
 using namespace process;
 
+using process::firewall::DisabledEndpointsFirewallRule;
+using process::firewall::FirewallRule;
+
 using process::network::Address;
 using process::network::Socket;
 
@@ -1880,3 +1884,155 @@ TEST(Process, PercentEncodedURLs)
   terminate(process);
   wait(process);
 }
+
+
+class HTTPEndpointProcess : public Process<HTTPEndpointProcess>
+{
+public:
+  explicit HTTPEndpointProcess(const std::string& id)
+    : ProcessBase(id) {}
+
+  virtual void initialize()
+  {
+    route(
+        "/handler1",
+        None(),
+        &HTTPEndpointProcess::handler1);
+    route(
+        "/handler2",
+        None(),
+        &HTTPEndpointProcess::handler2);
+    route(
+        "/handler3",
+        None(),
+        &HTTPEndpointProcess::handler3);
+  }
+
+  MOCK_METHOD1(handler1, Future<http::Response>(const http::Request&));
+  MOCK_METHOD1(handler2, Future<http::Response>(const http::Request&));
+  MOCK_METHOD1(handler3, Future<http::Response>(const http::Request&));
+};
+
+
+// Sets firewall rules which disable endpoints on a process and then
+// attempts to connect to those endpoints.
+TEST(Process, FirewallDisablePaths)
+{
+  const string processId = "testprocess";
+
+  hashset<string> endpoints;
+  endpoints.insert(path::join("", processId, "handler1"));
+  endpoints.insert(path::join("", processId, "handler2/nested"));
+  // Patterns are not supported, so this should do nothing.
+  endpoints.insert(path::join("", processId, "handler3/*"));
+
+  std::vector<Owned<FirewallRule>> rules;
+  rules.emplace_back(new DisabledEndpointsFirewallRule(endpoints));
+  process::firewall::install(std::move(rules));
+
+  HTTPEndpointProcess process(processId);
+
+  PID<HTTPEndpointProcess> pid = spawn(process);
+
+  // Test call to a disabled endpoint.
+  Future<http::Response> response = http::get(pid, "handler1");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[403], response.get().status);
+
+  // Test call to a non disabled endpoint.
+  // Substrings should not match.
+  EXPECT_CALL(process, handler2(_))
+    .WillOnce(Return(http::OK()));
+
+  response = http::get(pid, "handler2");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  // Test nested endpoints. Full paths needed for match.
+  response = http::get(pid, "handler2/nested");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[403], response.get().status);
+
+  EXPECT_CALL(process, handler2(_))
+    .WillOnce(Return(http::OK()));
+
+  response = http::get(pid, "handler2/nested/path");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  EXPECT_CALL(process, handler3(_))
+    .WillOnce(Return(http::OK()));
+
+  // Test a wildcard rule. Since they are not supported, it must have
+  // no effect at all.
+  response = http::get(pid, "handler3");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  EXPECT_CALL(process, handler3(_))
+    .WillOnce(Return(http::OK()));
+
+  response = http::get(pid, "handler3/nested");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  terminate(process);
+  wait(process);
+}
+
+
+// Test that firewall rules can be changed by changing the vector.
+// An empty vector should allow all paths.
+TEST(Process, FirewallUninstall)
+{
+  const string processId = "testprocess";
+
+  hashset<string> endpoints;
+  endpoints.insert(path::join("", processId, "handler1"));
+  endpoints.insert(path::join("", processId, "handler2"));
+
+  std::vector<Owned<FirewallRule>> rules;
+  rules.emplace_back(new DisabledEndpointsFirewallRule(endpoints));
+  process::firewall::install(std::move(rules));
+
+  HTTPEndpointProcess process(processId);
+
+  PID<HTTPEndpointProcess> pid = spawn(process);
+
+  Future<http::Response> response = http::get(pid, "handler1");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[403], response.get().status);
+
+  response = http::get(pid, "handler2");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[403], response.get().status);
+
+  process::firewall::install(std::vector<Owned<FirewallRule>>());
+
+  EXPECT_CALL(process, handler1(_))
+    .WillOnce(Return(http::OK()));
+
+  response = http::get(pid, "handler1");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  EXPECT_CALL(process, handler2(_))
+    .WillOnce(Return(http::OK()));
+
+  response = http::get(pid, "handler2");
+
+  AWAIT_READY(response);
+  EXPECT_EQ(http::statuses[200], response.get().status);
+
+  terminate(process);
+  wait(process);
+}


[2/2] mesos git commit: Added a flag which controls libprocess firewall initialzation.

Posted by ti...@apache.org.
Added a flag which controls libprocess firewall initialzation.

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


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

Branch: refs/heads/master
Commit: 3ef08fafd16909310b1b15b25168061c409b2144
Parents: 70c75c0
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Mon Jun 8 14:17:58 2015 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Mon Jun 8 14:17:58 2015 +0200

----------------------------------------------------------------------
 docs/configuration.md    | 23 +++++++++++++++
 src/Makefile.am          |  8 ++++++
 src/master/flags.cpp     | 19 +++++++++++++
 src/master/flags.hpp     |  3 ++
 src/master/main.cpp      | 21 ++++++++++++++
 src/messages/flags.hpp   | 65 +++++++++++++++++++++++++++++++++++++++++++
 src/messages/flags.proto | 31 +++++++++++++++++++++
 src/slave/flags.cpp      | 19 +++++++++++++
 src/slave/flags.hpp      |  3 ++
 src/slave/main.cpp       | 23 +++++++++++++++
 10 files changed, 215 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 7d6e786..aaf65bf 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -42,6 +42,29 @@ If you have special compilation requirements, please refer to `./configure --hel
   </tr>
   <tr>
     <td>
+      --firewall_rules=VALUE
+    </td>
+    <td>
+      The value could be a JSON formatted string of rules or a file path
+      containing the JSON formated rules used in the endpoints firewall. Path
+      could be of the form <code>file:///path/to/file</code> or
+      <code>/path/to/file</code>.
+      <p/>
+      See the Firewall message in flags.proto for the expected format.
+      <p/>
+      Example:
+<pre><code>{
+  "disabled_endpoints" : {
+    "paths" : [
+      "/files/browse.json",
+      "/slave(0)/stats.json",
+    ]
+  }
+}</code></pre>
+    </td>
+  </tr>
+  <tr>
+    <td>
       --[no-]help
     </td>
     <td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index ec7f41f..10b1902 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -195,6 +195,11 @@ PYTHON_PROTOS =								\
 BUILT_SOURCES += $(CXX_PROTOS) $(JAVA_PROTOS) $(PYTHON_PROTOS)
 CLEANFILES += $(CXX_PROTOS) $(JAVA_PROTOS) $(PYTHON_PROTOS)
 
+FLAGS_PROTOS = messages/flags.pb.cc messages/flags.pb.h
+
+BUILT_SOURCES += $(FLAGS_PROTOS)
+CLEANFILES += $(FLAGS_PROTOS)
+
 MESSAGES_PROTOS = messages/messages.pb.cc messages/messages.pb.h
 
 BUILT_SOURCES += $(MESSAGES_PROTOS)
@@ -335,6 +340,7 @@ noinst_LTLIBRARIES += libmesos_no_3rdparty.la
 
 nodist_libmesos_no_3rdparty_la_SOURCES =				\
   $(CXX_PROTOS)								\
+  $(FLAGS_PROTOS)							\
   $(MESSAGES_PROTOS)							\
   $(REGISTRY_PROTOS)
 
@@ -413,6 +419,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	zookeeper/zookeeper.cpp						\
 	zookeeper/authentication.cpp					\
 	zookeeper/group.cpp						\
+	messages/flags.proto						\
 	messages/messages.proto
 
 pkginclude_HEADERS =							\
@@ -605,6 +612,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/allocator/mesos/hierarchical.hpp				\
 	master/allocator/sorter/drf/sorter.hpp				\
 	master/allocator/sorter/sorter.hpp				\
+	messages/flags.hpp						\
 	messages/messages.hpp						\
 	module/manager.hpp						\
 	sched/constants.hpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index 49d953a..4377715 100644
--- a/src/master/flags.cpp
+++ b/src/master/flags.cpp
@@ -251,6 +251,25 @@ mesos::internal::master::Flags::Flags()
       "              ]\n"
       "}");
 
+  add(&Flags::firewall_rules,
+      "firewall_rules",
+      "The value could be a JSON formatted string of rules or a\n"
+      "file path containing the JSON formated rules used in the endpoints\n"
+      "firewall. Path must be of the form 'file:///path/to/file'\n"
+      "or '/path/to/file'.\n"
+      "\n"
+      "See the Firewall message in flags.proto for the expected format.\n"
+      "\n"
+      "Example:\n"
+      "{\n"
+      "  \"disabled_endpoints\" : {\n"
+      "    \"paths\" : [\n"
+      "      \"/files/browse.json\",\n"
+      "      \"/slave(0)/stats.json\",\n"
+      "    ]\n"
+      "  }\n"
+      "}");
+
   add(&Flags::rate_limits,
       "rate_limits",
       "The value could be a JSON formatted string of rate limits\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 84fa238..55ed3a9 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -30,6 +30,8 @@
 
 #include "logging/flags.hpp"
 
+#include "messages/flags.hpp"
+
 namespace mesos {
 namespace internal {
 namespace master {
@@ -64,6 +66,7 @@ public:
   bool authenticate_slaves;
   Option<Path> credentials;
   Option<ACLs> acls;
+  Option<Firewall> firewall_rules;
   Option<RateLimits> rate_limits;
   Option<Duration> offer_timeout;
   Option<Modules> modules;

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index 3d490c3..1c33e3b 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -89,6 +89,9 @@ using process::Owned;
 using process::RateLimiter;
 using process::UPID;
 
+using process::firewall::DisabledEndpointsFirewallRule;
+using process::firewall::FirewallRule;
+
 using std::cerr;
 using std::cout;
 using std::endl;
@@ -340,6 +343,24 @@ int main(int argc, char** argv)
     slaveRemovalLimiter = new RateLimiter(permits.get(), duration.get());
   }
 
+  if (flags.firewall_rules.isSome()) {
+    const Firewall rules = flags.firewall_rules.get();
+
+    std::vector<Owned<FirewallRule>> _rules;
+
+    if (rules.has_disabled_endpoints()) {
+      hashset<string> paths;
+
+      for (int i = 0; i < rules.disabled_endpoints().paths_size(); ++i) {
+        paths.insert(rules.disabled_endpoints().paths(i));
+      }
+
+      _rules.emplace_back(new DisabledEndpointsFirewallRule(paths));
+    }
+
+    process::firewall::install(std::move(_rules));
+  }
+
   // Create anonymous modules.
   foreach (const string& name, ModuleManager::find<Anonymous>()) {
     Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/messages/flags.hpp
----------------------------------------------------------------------
diff --git a/src/messages/flags.hpp b/src/messages/flags.hpp
new file mode 100644
index 0000000..41be419
--- /dev/null
+++ b/src/messages/flags.hpp
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MESSAGES_FLAGS_HPP__
+#define __MESSAGES_FLAGS_HPP__
+
+#include <string>
+
+#include <stout/error.hpp>
+#include <stout/json.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/try.hpp>
+
+#include <stout/flags/parse.hpp>
+
+#include "common/parse.hpp"
+
+#include "messages/flags.pb.h"
+
+namespace flags {
+
+template <>
+inline Try<mesos::internal::Firewall> parse(const std::string& value)
+{
+  // Convert from string or file to JSON.
+  Try<JSON::Object> json = parse<JSON::Object>(value);
+  if (json.isError()) {
+    return Error(json.error());
+  }
+
+  // Convert from JSON to Protobuf.
+  return protobuf::parse<mesos::internal::Firewall>(json.get());
+}
+
+} // namespace flags {
+
+namespace mesos {
+namespace internal {
+
+inline std::ostream& operator << (
+    std::ostream& stream,
+    const Firewall& rules)
+{
+  return stream << rules.DebugString();
+}
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESSAGES_FLAGS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/messages/flags.proto
----------------------------------------------------------------------
diff --git a/src/messages/flags.proto b/src/messages/flags.proto
new file mode 100644
index 0000000..5400c92
--- /dev/null
+++ b/src/messages/flags.proto
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import "mesos/mesos.proto";
+
+package mesos.internal;
+
+// Initializes firewall rules to allow access control of the
+// libprocess endpoints.
+message Firewall {
+  message DisabledEndpointsRule {
+    repeated string paths = 1;
+  }
+
+  optional DisabledEndpointsRule disabled_endpoints = 1;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 1ae106e..99142fb 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -263,6 +263,25 @@ mesos::internal::slave::Flags::Flags()
       true);
 #endif
 
+  add(&Flags::firewall_rules,
+      "firewall_rules",
+      "The value could be a JSON formatted string of rules or a\n"
+      "file path containing the JSON formated rules used in the endpoints\n"
+      "firewall. Path must be of the form 'file:///path/to/file'\n"
+      "or '/path/to/file'.\n"
+      "\n"
+      "See the Firewall message in flags.proto for the expected format.\n"
+      "\n"
+      "Example:\n"
+      "{\n"
+      "  \"disabled_endpoints\" : {\n"
+      "    \"paths\" : [\n"
+      "      \"/files/browse.json\",\n"
+      "      \"/slave(0)/stats.json\",\n"
+      "    ]\n"
+      "  }\n"
+      "}");
+
   add(&Flags::credential,
       "credential",
       "Either a path to a text with a single line\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 32d36ac..6c24e56 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -30,6 +30,8 @@
 
 #include "logging/flags.hpp"
 
+#include "messages/flags.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -78,6 +80,7 @@ public:
   Duration perf_duration;
   bool revocable_cpu_low_priority;
 #endif
+  Option<Firewall> firewall_rules;
   Option<Path> credential;
   Option<std::string> containerizer_path;
   std::string containerizers;

http://git-wip-us.apache.org/repos/asf/mesos/blob/3ef08faf/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index af090ae..c379243 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -26,6 +26,7 @@
 
 #include <stout/check.hpp>
 #include <stout/flags.hpp>
+#include <stout/hashset.hpp>
 #include <stout/nothing.hpp>
 #include <stout/os.hpp>
 #include <stout/stringify.hpp>
@@ -39,6 +40,7 @@
 
 #include "master/detector.hpp"
 
+#include "messages/flags.hpp"
 #include "messages/messages.hpp"
 
 #include "module/manager.hpp"
@@ -58,6 +60,9 @@ using mesos::slave::ResourceEstimator;
 
 using mesos::SlaveInfo;
 
+using process::firewall::DisabledEndpointsFirewallRule;
+using process::firewall::FirewallRule;
+
 using std::cerr;
 using std::cout;
 using std::endl;
@@ -174,6 +179,24 @@ int main(int argc, char** argv)
       << "Failed to create a master detector: " << detector.error();
   }
 
+  if (flags.firewall_rules.isSome()) {
+    const Firewall rules = flags.firewall_rules.get();
+
+    std::vector<Owned<FirewallRule>> _rules;
+
+    if (rules.has_disabled_endpoints()) {
+      hashset<string> paths;
+
+      for (int i = 0; i < rules.disabled_endpoints().paths_size(); ++i) {
+        paths.insert(rules.disabled_endpoints().paths(i));
+      }
+
+      _rules.emplace_back(new DisabledEndpointsFirewallRule(paths));
+    }
+
+    process::firewall::install(std::move(_rules));
+  }
+
   // Create anonymous modules.
   foreach (const string& name, ModuleManager::find<Anonymous>()) {
     Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);