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/05/25 00:32:46 UTC

[1/2] mesos git commit: Renamed an ambiguous variable in the `network/ports` isolator.

Repository: mesos
Updated Branches:
  refs/heads/master d1207a151 -> 24359e643


Renamed an ambiguous variable in the `network/ports` isolator.

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


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

Branch: refs/heads/master
Commit: 4f95d751b0e890bbc6ede3d2adc8d920c6b1c714
Parents: d1207a1
Author: Xudong Ni <xd...@yahoo.com>
Authored: Thu May 24 17:06:34 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu May 24 17:10:42 2018 -0700

----------------------------------------------------------------------
 .../containerizer/mesos/isolators/network/ports.cpp    | 13 ++++++++-----
 .../containerizer/mesos/isolators/network/ports.hpp    |  2 +-
 2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4f95d751/src/slave/containerizer/mesos/isolators/network/ports.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.cpp b/src/slave/containerizer/mesos/isolators/network/ports.cpp
index 1f84ed4..53e894d 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.cpp
@@ -522,12 +522,13 @@ Future<Nothing> NetworkPortsIsolatorProcess::update(
   Option<Value::Ranges> ports = resources.ports();
   if (ports.isSome()) {
     const Owned<Info>& info = infos.at(containerId);
-    info->ports = rangesToIntervalSet<uint16_t>(ports.get()).get();
+    info->allocatedPorts = rangesToIntervalSet<uint16_t>(ports.get()).get();
   } else {
-    info->ports = IntervalSet<uint16_t>();
+    info->allocatedPorts = IntervalSet<uint16_t>();
   }
 
-  LOG(INFO) << "Updated ports to " << intervalSetToRanges(info->ports.get())
+  LOG(INFO) << "Updated ports to "
+            << intervalSetToRanges(info->allocatedPorts.get())
             << " for container " << containerId;
 
   return Nothing();
@@ -568,8 +569,10 @@ Future<Nothing> NetworkPortsIsolatorProcess::check(
     // for this container.
     const Owned<Info>& info = infos.at(rootContainerId);
 
-    if (info->ports.isSome() && !info->ports->contains(ports)) {
-      const IntervalSet<uint16_t> unallocatedPorts = ports - info->ports.get();
+    if (info->allocatedPorts.isSome() &&
+        !info->allocatedPorts->contains(ports)) {
+      const IntervalSet<uint16_t> unallocatedPorts =
+          ports - info->allocatedPorts.get();
 
       Resource resource;
       resource.set_name("ports");

http://git-wip-us.apache.org/repos/asf/mesos/blob/4f95d751/src/slave/containerizer/mesos/isolators/network/ports.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.hpp b/src/slave/containerizer/mesos/isolators/network/ports.hpp
index ba71087..d9a97d1 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.hpp
@@ -91,7 +91,7 @@ private:
 
   struct Info
   {
-    Option<IntervalSet<uint16_t>> ports;
+    Option<IntervalSet<uint16_t>> allocatedPorts;
     process::Promise<mesos::slave::ContainerLimitation> limitation;
   };
 


[2/2] mesos git commit: Added enforce_container_ports flag for `network/ports` isolation.

Posted by jp...@apache.org.
Added enforce_container_ports flag for `network/ports` isolation.

To reduce deployment risk, a no-enforce mode is added for the
`network/ports` isolator. When this flag is false (which is the
default) the isolator will not raise any limitation, even if the
task uses ports not in its container resources.

Added new test for this flag and updated the existing tests.

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


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

Branch: refs/heads/master
Commit: 24359e64323e65e90e07b485840b334968307f15
Parents: 4f95d75
Author: Xudong Ni <xd...@yahoo.com>
Authored: Thu May 24 17:06:40 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu May 24 17:12:00 2018 -0700

----------------------------------------------------------------------
 docs/configuration/agent.md                     |   9 ++
 docs/isolators/network-ports.md                 |   6 +
 .../mesos/isolators/network/ports.cpp           |  25 +++-
 .../mesos/isolators/network/ports.hpp           |   3 +
 src/slave/flags.cpp                             |   5 +
 src/slave/flags.hpp                             |   1 +
 .../containerizer/ports_isolator_tests.cpp      | 126 +++++++++++++++++++
 7 files changed, 170 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/docs/configuration/agent.md
----------------------------------------------------------------------
diff --git a/docs/configuration/agent.md b/docs/configuration/agent.md
index 962211a..af0c040 100644
--- a/docs/configuration/agent.md
+++ b/docs/configuration/agent.md
@@ -662,6 +662,15 @@ is used by the <code>disk/du</code> and <code>disk/xfs</code> isolators. (defaul
 </tr>
 <tr>
   <td>
+    --[no-]enforce_container_ports
+  </td>
+  <td>
+Whether to enable network port enforcement for containers. This flag
+is used by the <code>network/ports</code> isolator. (default: false)
+  </td>
+</tr>
+<tr>
+  <td>
     --executor_environment_variables=VALUE
   </td>
   <td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/docs/isolators/network-ports.md
----------------------------------------------------------------------
diff --git a/docs/isolators/network-ports.md b/docs/isolators/network-ports.md
index ea63968..5d14fc2 100644
--- a/docs/isolators/network-ports.md
+++ b/docs/isolators/network-ports.md
@@ -41,6 +41,12 @@ to use the Linux launcher (i.e. the agent has the `--launcher=linux`
 flag), because it uses Linux cgroups to track the processes belonging
 to a Mesos task.
 
+The `--enforce_container_ports` flag, specifies whether the network
+ports isolator should terminate tasks that listen on ports they have
+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.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/src/slave/containerizer/mesos/isolators/network/ports.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.cpp b/src/slave/containerizer/mesos/isolators/network/ports.cpp
index 53e894d..82c1441 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.cpp
@@ -343,6 +343,7 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
       new NetworkPortsIsolatorProcess(
           strings::contains(flags.isolation, "network/cni"),
           flags.container_ports_watch_interval,
+          flags.enforce_container_ports,
           flags.cgroups_root,
           freezerHierarchy.get(),
           agentPorts)));
@@ -352,12 +353,14 @@ Try<Isolator*> NetworkPortsIsolatorProcess::create(const Flags& flags)
 NetworkPortsIsolatorProcess::NetworkPortsIsolatorProcess(
     bool _cniIsolatorEnabled,
     const Duration& _watchInterval,
+    const bool& _enforceContainerPorts,
     const string& _cgroupsRoot,
     const string& _freezerHierarchy,
     const Option<IntervalSet<uint16_t>>& _agentPorts)
   : ProcessBase(process::ID::generate("network-ports-isolator")),
     cniIsolatorEnabled(_cniIsolatorEnabled),
     watchInterval(_watchInterval),
+    enforceContainerPorts(_enforceContainerPorts),
     cgroupsRoot(_cgroupsRoot),
     freezerHierarchy(_freezerHierarchy),
     agentPorts(_agentPorts)
@@ -574,6 +577,16 @@ Future<Nothing> NetworkPortsIsolatorProcess::check(
       const IntervalSet<uint16_t> unallocatedPorts =
           ports - info->allocatedPorts.get();
 
+      // Only log unallocated ports once to prevent excessive logging
+      // for the same unallocated ports while port enforcement is disabled.
+      if (info->activePorts.isSome() && info->activePorts == ports) {
+        continue;
+      }
+
+      // Cache the last listeners port sample so that we will
+      // only log new ports resource violations.
+      info->activePorts = ports;
+
       Resource resource;
       resource.set_name("ports");
       resource.set_type(Value::RANGES);
@@ -587,11 +600,13 @@ Future<Nothing> NetworkPortsIsolatorProcess::check(
 
       LOG(INFO) << message;
 
-      info->limitation.set(
-          protobuf::slave::createContainerLimitation(
-              Resources(resource),
-              message,
-              TaskStatus::REASON_CONTAINER_LIMITATION));
+      if (enforceContainerPorts) {
+        info->limitation.set(
+        protobuf::slave::createContainerLimitation(
+            Resources(resource),
+            message,
+            TaskStatus::REASON_CONTAINER_LIMITATION));
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/src/slave/containerizer/mesos/isolators/network/ports.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/ports.hpp b/src/slave/containerizer/mesos/isolators/network/ports.hpp
index d9a97d1..8d467ce 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.hpp
@@ -85,6 +85,7 @@ private:
   NetworkPortsIsolatorProcess(
       bool _cniIsolatorEnabled,
       const Duration& _watchInterval,
+      const bool& _enforcePortsEnabled,
       const std::string& _cgroupsRoot,
       const std::string& _freezerHierarchy,
       const Option<IntervalSet<uint16_t>>& agentPorts);
@@ -92,11 +93,13 @@ private:
   struct Info
   {
     Option<IntervalSet<uint16_t>> allocatedPorts;
+    Option<IntervalSet<uint16_t>> activePorts;
     process::Promise<mesos::slave::ContainerLimitation> limitation;
   };
 
   const bool cniIsolatorEnabled;
   const Duration watchInterval;
+  const bool enforceContainerPorts;
   const std::string cgroupsRoot;
   const std::string freezerHierarchy;
   const Option<IntervalSet<uint16_t>> agentPorts;

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index a319b5e..23d9bb1 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -1117,6 +1117,11 @@ 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.",
+      false);
 #endif // ENABLE_NETWORK_PORTS_ISOLATOR
 
   add(&Flags::network_cni_plugins_dir,

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index a839591..ae09e19 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -152,6 +152,7 @@ public:
 #ifdef ENABLE_NETWORK_PORTS_ISOLATOR
   Duration container_ports_watch_interval;
   bool check_agent_port_range_only;
+  bool enforce_container_ports;
 #endif // ENABLE_NETWORK_PORTS_ISOLATOR
 
   Option<std::string> network_cni_plugins_dir;

http://git-wip-us.apache.org/repos/asf/mesos/blob/24359e64/src/tests/containerizer/ports_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/ports_isolator_tests.cpp b/src/tests/containerizer/ports_isolator_tests.cpp
index c5b9f92..db080c4 100644
--- a/src/tests/containerizer/ports_isolator_tests.cpp
+++ b/src/tests/containerizer/ports_isolator_tests.cpp
@@ -250,6 +250,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_CommandExecutorPorts)
   flags.isolation = "network/ports";
   flags.launcher = "linux";
   flags.check_agent_port_range_only = false;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -343,6 +344,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_AllocatedPorts)
   // Watch only the agent ports resources range because we want this
   // test to trigger on the nc command, not on the command executor.
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -496,6 +498,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_NoPortsResource)
   // Watch only the agent ports resources range because we want this
   // test to trigger on the nc command, not on the command executor.
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -615,6 +618,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_DefaultPortsResource)
   // Watch only the agent ports resources range because we want this
   // test to trigger on the nc command, not on the command executor.
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -726,6 +730,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_UnallocatedPorts)
   // Watch only the agent ports resources range because we want this
   // test to trigger on the nc command, not on the command executor.
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -824,6 +829,122 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_UnallocatedPorts)
 }
 
 
+// This test verifies that a task that listens on a port for which
+// it has no resources is detected and will not be killed by
+// a container limitation if enforce_container_ports is false.
+TEST_F(NetworkPortsIsolatorTest, ROOT_NC_NoPortEnforcement)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "network/ports";
+  flags.launcher = "linux";
+
+  // Watch only the agent ports resources range because we want this
+  // test to trigger on the nc command, not on the command executor.
+  flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = false;
+
+  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 = selectOtherPort(resources, taskPort);
+
+  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 container ports are not being enforced, 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)
@@ -920,6 +1041,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverBadTask)
   // and terminate it.
   flags.isolation = "network/ports";
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
@@ -1044,6 +1166,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverGoodTask)
   // listening and will let it continue running.
   flags.isolation = "network/ports";
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Future<SlaveReregisteredMessage> slaveReregisteredMessage =
     FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);
@@ -1100,6 +1223,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_TaskGroup)
   flags.isolation = "network/ports";
   flags.launcher = "linux";
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
@@ -1359,6 +1483,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverNestedBadTask)
   // and terminate it.
   flags.isolation = "network/ports";
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   slave = cluster::Slave::create(detector.get(), flags, slaveId);
   ASSERT_SOME(slave);
@@ -1536,6 +1661,7 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverNestedGoodTask)
   // listening and will let it continue running.
   flags.isolation = "network/ports";
   flags.check_agent_port_range_only = true;
+  flags.enforce_container_ports = true;
 
   slave = cluster::Slave::create(detector.get(), flags, slaveId);
   ASSERT_SOME(slave);