You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jp...@apache.org on 2018/08/25 00:18:56 UTC

[mesos] branch master updated: Added a custom port range option to the `network/ports` isolator.

This is an automated email from the ASF dual-hosted git repository.

jpeach pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git


The following commit(s) were added to refs/heads/master by this push:
     new 53ef890  Added a custom port range option to the `network/ports` isolator.
53ef890 is described below

commit 53ef8902369b63d412f26ff8abe36d330e346fae
Author: Xudong Ni <xd...@yahoo.com>
AuthorDate: Fri Aug 24 16:26:54 2018 -0700

    Added a custom port range option to the `network/ports` isolator.
    
    Added the `--container_ports_isolated_range` flag to the
    `network/ports` isolator. This allows the operator to specify a custom
    port range to be protected by the isolator. If a task listens on a port
    that it isn't holding resources for, the isolator will not raise a
    limitation unless the port is within this range. We can represent the
    `--check_agent_port_range_only` as a special case of a protected range.
    
    Review: https://reviews.apache.org/r/68366/
---
 docs/configuration/agent.md                        |  19 ++-
 docs/isolators/network-ports.md                    |  16 ++-
 .../mesos/isolators/network/ports.cpp              |  69 ++++++++--
 .../mesos/isolators/network/ports.hpp              |   4 +-
 src/slave/flags.cpp                                |  10 +-
 src/slave/flags.hpp                                |   1 +
 src/tests/containerizer/ports_isolator_tests.cpp   | 149 ++++++++++++++++++++-
 7 files changed, 244 insertions(+), 24 deletions(-)

diff --git a/docs/configuration/agent.md b/docs/configuration/agent.md
index e98a978..e45b1d2 100644
--- a/docs/configuration/agent.md
+++ b/docs/configuration/agent.md
@@ -406,11 +406,12 @@ Name of the root cgroup. (default: mesos)
     --[no-]check_agent_port_range_only
   </td>
   <td>
-When this is true, the `network/ports` isolator allows tasks to
+When this is true, the <code>network/ports</code> isolator allows tasks to
 listen on additional ports provided they fall outside the range
 published by the agent's resources. Otherwise tasks are restricted
 to only listen on ports for which they have been assigned resources.
-(default: false)
+(default: false); This flag can't be used in conjunction with
+<code>--container_ports_isolated_range</code>.
   </td>
 </tr>
 
@@ -436,11 +437,23 @@ in the sandbox directory.
   </td>
 </tr>
 
+<tr id="container_ports_isolated_range">
+  <td>
+    --container_ports_isolated_range=VALUE
+  </td>
+  <td>
+When this flag is set, <code>network/ports</code> isolator will only enforce
+the port isolation for the given range of ports range. This flag can't
+be used in conjunction with <code>--check_agent_port_range_only</code>.
+Example: <code>[0-35000]</code>
+  </td>
+</tr>
+
 <tr id="container_ports_watch_interval">
   <td>
     --container_ports_watch_interval=VALUE
   </td>
-Interval at which the `network/ports` isolator should check for
+Interval at which the <code>network/ports</code> isolator should check for
 containers listening on ports they don't have resources for.
 (default: 30secs)
   <td>
diff --git a/docs/isolators/network-ports.md b/docs/isolators/network-ports.md
index 5d14fc2..de4eebe 100644
--- a/docs/isolators/network-ports.md
+++ b/docs/isolators/network-ports.md
@@ -47,16 +47,24 @@ not been assigned. If enforcement is disabled, the isolator will log
 violations but will not terminate tasks. By default, network port
 enforcement is disabled.
 
-The `--container_ports_watch_interval` flag specifies the interval
-between task port reconciliations.
-
 If the `--check_agent_port_range_only` flag is specified, the isolator
 will not kill tasks that listen on unallocated ports outside the range
 of port resources the agent offers to tasks. This flag is required when
 using the default Mesos executors or any custom executor that uses the
 native Mesos Java or Python bindings since the native Mesos libraries
 will always implicity listen on a socket. This flag should not be
-required for custom executors that use the HTTP executor API.
+required for custom executors that use the HTTP executor API. This flag
+can't be used in conjunction with `--container_ports_isolated_range`.
+
+The `--container_ports_isolated_range` works just like the
+`--check_agent_port_range_only` flag except that it accepts an
+arbitrary range of ports. When this range flag is set, the isolator
+will neither log nor terminate the task if the used unallocated
+ports that are outside this range. This flag can't be used in
+conjunction with `--check_agent_port_range_only`.
+
+The `--container_ports_watch_interval` flag specifies the interval
+between task port reconciliations.
 
 The network ports isolator ignores tasks that belong to a [CNI](../cni.md)
 network since these tasks do not share the host network namespace.
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.cpp b/src/slave/containerizer/mesos/isolators/network/ports.cpp
index 2a7ff25..fec74d3 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.cpp
@@ -74,7 +74,7 @@ static hashmap<ContainerID, IntervalSet<uint16_t>>
 collectContainerListeners(
     const string& cgroupsRoot,
     const string& freezerHierarchy,
-    const Option<IntervalSet<uint16_t>>& agentPorts,
+    const Option<IntervalSet<uint16_t>>& isolatedPorts,
     const hashset<ContainerID>& containerIds)
 {
   hashmap<ContainerID, IntervalSet<uint16_t>> listeners;
@@ -147,9 +147,9 @@ collectContainerListeners(
           }
         }
 
-        // If we are filtering by agent ports, then we only collect this
-        // listen socket if it falls within the agent port range.
-        if (agentPorts.isNone() || agentPorts->contains(address.port)) {
+        // Only collect this listen socket if it falls within the
+        // isolated range.
+        if (isolatedPorts.isNone() || isolatedPorts->contains(address.port)) {
           listeners[containerId].add(address.port);
         }
       }
@@ -280,6 +280,13 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
     return Error("The 'network/ports' isolator requires the 'linux' launcher");
   }
 
+  if (flags.check_agent_port_range_only &&
+      flags.container_ports_isolated_range.isSome()) {
+    return Error(
+        "Only one of `--check_agent_port_range_only` or "
+        "'--container_ports_isolated_range` should be specified");
+  }
+
   Try<string> freezerHierarchy = cgroups::prepare(
       flags.cgroups_hierarchy,
       "freezer",
@@ -291,8 +298,8 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
         freezerHierarchy.error());
   }
 
-
-  Option<IntervalSet<uint16_t>> agentPorts = None();
+  // Set None as the default of isolated ports range.
+  Option<IntervalSet<uint16_t>> isolatedPorts = None();
 
   // If we are only watching the ports in the agent resources, figure
   // out what the agent ports will be by checking the resources flag
@@ -318,7 +325,7 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
               stringify(DEFAULT_PORTS),
               flags.default_role).get());
 
-      agentPorts =
+      isolatedPorts =
         rangesToIntervalSet<uint16_t>(resources->ports().get()).get();
     } else if (resources->ports().isSome()) {
       // Use the given non-empty ports resource.
@@ -332,11 +339,47 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
             "': " + ports.error());
       }
 
-      agentPorts = ports.get();
+      isolatedPorts = ports.get();
     } else {
       // An empty ports resource was specified.
-      agentPorts = IntervalSet<uint16_t>{};
+      isolatedPorts = IntervalSet<uint16_t>{};
+    }
+  }
+
+  // Use the given isolated ports range if specified.
+  if (flags.container_ports_isolated_range.isSome()) {
+    Try<Resource> portRange =
+      Resources::parse(
+          "ports",
+          flags.container_ports_isolated_range.get(),
+          "*");
+
+    if (portRange.isError()) {
+      return Error(
+          "Failed to parse isolated ports range '" +
+          flags.container_ports_isolated_range.get() + "'");
     }
+
+    if (portRange->type() != Value::RANGES) {
+      return Error(
+          "Invalid port range resource type " +
+          mesos::Value_Type_Name(portRange->type()) +
+          ", expecting " +
+          mesos::Value_Type_Name(Value::RANGES));
+    }
+
+    Try<IntervalSet<uint16_t>> ports =
+      rangesToIntervalSet<uint16_t>(portRange->ranges());
+
+    if (ports.isError()) {
+      return Error(ports.error());
+    }
+
+    isolatedPorts = ports.get();
+  }
+
+  if (isolatedPorts.isSome()) {
+    LOG(INFO) << "Isolating port range " << stringify(isolatedPorts.get());
   }
 
   return new MesosIsolator(process::Owned<MesosIsolatorProcess>(
@@ -346,7 +389,7 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
           flags.enforce_container_ports,
           flags.cgroups_root,
           freezerHierarchy.get(),
-          agentPorts)));
+          isolatedPorts)));
 }
 
 
@@ -356,14 +399,14 @@ NetworkPortsIsolatorProcess::NetworkPortsIsolatorProcess(
     const bool& _enforceContainerPorts,
     const string& _cgroupsRoot,
     const string& _freezerHierarchy,
-    const Option<IntervalSet<uint16_t>>& _agentPorts)
+    const Option<IntervalSet<uint16_t>>& _isolatedPorts)
   : ProcessBase(process::ID::generate("network-ports-isolator")),
     cniIsolatorEnabled(_cniIsolatorEnabled),
     watchInterval(_watchInterval),
     enforceContainerPorts(_enforceContainerPorts),
     cgroupsRoot(_cgroupsRoot),
     freezerHierarchy(_freezerHierarchy),
-    agentPorts(_agentPorts)
+    isolatedPorts(_isolatedPorts)
 {
 }
 
@@ -631,7 +674,7 @@ void NetworkPortsIsolatorProcess::initialize()
             &collectContainerListeners,
             cgroupsRoot,
             freezerHierarchy,
-            agentPorts,
+            isolatedPorts,
             infos.keys())
           .then(defer(self, &NetworkPortsIsolatorProcess::check, lambda::_1))
           .then([]() -> ControlFlow<Nothing> { return Continue(); });
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.hpp b/src/slave/containerizer/mesos/isolators/network/ports.hpp
index 6944d01..c0d0858 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.hpp
@@ -88,7 +88,7 @@ private:
       const bool& _enforcePortsEnabled,
       const std::string& _cgroupsRoot,
       const std::string& _freezerHierarchy,
-      const Option<IntervalSet<uint16_t>>& agentPorts);
+      const Option<IntervalSet<uint16_t>>& isolatedPorts);
 
   struct Info
   {
@@ -102,7 +102,7 @@ private:
   const bool enforceContainerPorts;
   const std::string cgroupsRoot;
   const std::string freezerHierarchy;
-  const Option<IntervalSet<uint16_t>> agentPorts;
+  const Option<IntervalSet<uint16_t>> isolatedPorts;
 
   hashmap<ContainerID, process::Owned<Info>> infos;
 };
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index e017f39..fd53d90 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -1146,11 +1146,19 @@ mesos::internal::slave::Flags::Flags()
       "published by the agent's resources. Otherwise tasks are restricted\n"
       "to only listen on ports for which they have been assigned resources.",
       false);
+
   add(&Flags::enforce_container_ports,
       "enforce_container_ports",
       "Whether to enable port enforcement for containers. This flag\n"
-      "is used by `network/ports` isolator.",
+      "is used by the `network/ports` isolator.",
       false);
+
+  add(&Flags::container_ports_isolated_range,
+      "container_ports_isolated_range",
+      "When this flag is specified, the `network/ports` isolator will\n"
+      "only enforce port isolation for the specified range of ports.\n"
+      "(Example: `[0-35000]`)\n");
+
 #endif // ENABLE_NETWORK_PORTS_ISOLATOR
 
   add(&Flags::network_cni_plugins_dir,
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index bff194f..f6db2bb 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -156,6 +156,7 @@ public:
   Duration container_ports_watch_interval;
   bool check_agent_port_range_only;
   bool enforce_container_ports;
+  Option<std::string> container_ports_isolated_range;
 #endif // ENABLE_NETWORK_PORTS_ISOLATOR
 
   Option<std::string> network_cni_plugins_dir;
diff --git a/src/tests/containerizer/ports_isolator_tests.cpp b/src/tests/containerizer/ports_isolator_tests.cpp
index db080c4..24c1f07 100644
--- a/src/tests/containerizer/ports_isolator_tests.cpp
+++ b/src/tests/containerizer/ports_isolator_tests.cpp
@@ -216,7 +216,8 @@ TEST(NetworkPortsIsolatorUtilityTest, QueryProcessSockets)
 
 
 // This test verifies that the `network/ports` isolator throws
-// an error unless the `linux` launcher is being used.
+// an error unless the `linux` launcher is being used, and also
+// verifies the input format of the isolated ports range.
 TEST_F(NetworkPortsIsolatorTest, ROOT_IsolatorFlags)
 {
   StandaloneMasterDetector detector;
@@ -233,6 +234,27 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_IsolatorFlags)
   flags.launcher = "linux";
   slave = StartSlave(&detector, flags);
   ASSERT_SOME(slave);
+
+  flags.enforce_container_ports = true;
+
+  // Ports are 16 bit.
+  flags.container_ports_isolated_range = "[100-65536]";
+  slave = StartSlave(&detector, flags);
+  EXPECT_ERROR(slave);
+
+  // Ports must be a range.
+  flags.container_ports_isolated_range = "foo";
+  slave = StartSlave(&detector, flags);
+  EXPECT_ERROR(slave);
+
+  // Ports must be a range.
+  flags.container_ports_isolated_range = "100";
+  slave = StartSlave(&detector, flags);
+  EXPECT_ERROR(slave);
+
+  flags.container_ports_isolated_range = "[31000-32000]";
+  slave = StartSlave(&detector, flags);
+  ASSERT_SOME(slave);
 }
 
 
@@ -945,6 +967,131 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_NoPortEnforcement)
 }
 
 
+// This test verifies that a task that listens on a port which is not
+// in the isolated ports range will not be killed by a container
+// limitation while enforce_container_ports is true.
+TEST_F(NetworkPortsIsolatorTest, ROOT_NC_PortEnforcementIsolatedPort)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "network/ports";
+  flags.launcher = "linux";
+
+  // Watch only the isolated ports range because we want this test to
+  // show that invalid port usage out of the isolated range is allowed
+  flags.enforce_container_ports = true;
+  flags.container_ports_isolated_range = "[45000-45002]";
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_EQ(1u, offers->size());
+
+  const Offer& offer = offers.get()[0];
+
+  // Make sure we have a `ports` resource.
+  Resources resources(offer.resources());
+  ASSERT_SOME(resources.ports());
+  ASSERT_LE(1, resources.ports()->range().size());
+
+  uint16_t taskPort = selectRandomPort(resources);
+  uint16_t usedPort;
+
+  // We need to use a port that is inside the offered resources but outside
+  // the isolated range and not the same as the one we are accepting from
+  // the offer.
+  do {
+    usedPort = selectOtherPort(resources, taskPort);
+  } while (usedPort >= 45000 && usedPort <= 45002);
+
+  CHECK_NE(taskPort, usedPort);
+
+  resources = Resources::parse(
+      "cpus:1;mem:32;"
+      "ports:[" + stringify(taskPort) + "," + stringify(taskPort) + "]").get();
+
+  // Launch a task that uses a port that it hasn't been allocated.  Use
+  // "nc -k" so nc keeps running after accepting the healthcheck connection.
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      resources,
+      "nc -k -l " + stringify(usedPort));
+
+  addTcpHealthCheck(task, usedPort);
+
+  Future<TaskStatus> startingStatus;
+  Future<TaskStatus> runningStatus;
+  Future<TaskStatus> healthStatus;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&startingStatus))
+    .WillOnce(FutureArg<1>(&runningStatus))
+    .WillOnce(FutureArg<1>(&healthStatus));
+
+  driver.launchTasks(offer.id(), {task});
+
+  awaitStatusUpdateAcked(startingStatus);
+  EXPECT_EQ(task.task_id(), startingStatus->task_id());
+  EXPECT_EQ(TASK_STARTING, startingStatus->state());
+
+  awaitStatusUpdateAcked(runningStatus);
+  EXPECT_EQ(task.task_id(), runningStatus->task_id());
+  EXPECT_EQ(TASK_RUNNING, runningStatus->state());
+
+  awaitStatusUpdateAcked(healthStatus);
+  ASSERT_EQ(task.task_id(), healthStatus->task_id());
+  expectHealthyStatus(healthStatus.get());
+
+  Future<Nothing> check =
+    FUTURE_DISPATCH(_, &NetworkPortsIsolatorProcess::check);
+
+  Clock::pause();
+  Clock::advance(flags.container_ports_watch_interval);
+
+  AWAIT_READY(check);
+
+  Clock::settle();
+  Clock::resume();
+
+  // Since the container listening port is not in the isolated port range,
+  // we expect that the task should still be running after the check
+  // and that we should be able to explicitly kill it.
+  Future<TaskStatus> killedStatus;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&killedStatus));
+
+  driver.killTask(task.task_id());
+
+  AWAIT_READY(killedStatus);
+  EXPECT_EQ(task.task_id(), killedStatus->task_id());
+  EXPECT_EQ(TASK_KILLED, killedStatus->state());
+
+  driver.stop();
+  driver.join();
+}
+
+
 // Test that after we recover a task, the isolator notices that it
 // is using the wrong ports and kills it.
 TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverBadTask)