You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gr...@apache.org on 2017/12/16 02:07:47 UTC

mesos git commit: Toggled the authenticate_http_readwrite flag in MesosTest.

Repository: mesos
Updated Branches:
  refs/heads/master a4b1134e4 -> 91ea75e83


Toggled the authenticate_http_readwrite flag in MesosTest.

The default executor authentication mechanism depends on SSL support
being enabled in the build. Multiple tests that invoke the default
executor have compilation time checks for SSL support so they
can disable the Mesos agent authenticate_http_readwrite flag in
this case.  Rather than duplicating this check to multiple tests,
set it to an appropriate default in the MesosTest fixture.

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


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

Branch: refs/heads/master
Commit: 91ea75e83560dfa3a4a38563b942ef7ec2a5bb29
Parents: a4b1134
Author: James Peach <jp...@apache.org>
Authored: Thu Dec 14 11:24:52 2017 -0800
Committer: Greg Mann <gr...@gmail.com>
Committed: Fri Dec 15 16:29:50 2017 -0800

----------------------------------------------------------------------
 src/tests/api_tests.cpp                         |  4 +++
 src/tests/check_tests.cpp                       |  6 ----
 src/tests/container_daemon_tests.cpp            |  2 --
 .../environment_secret_isolator_tests.cpp       |  8 -----
 .../linux_capabilities_isolator_tests.cpp       |  8 -----
 .../containerizer/nvidia_gpu_isolator_tests.cpp |  8 -----
 .../containerizer/ports_isolator_tests.cpp      | 24 --------------
 .../posix_rlimits_isolator_tests.cpp            |  8 -----
 .../volume_host_path_isolator_tests.cpp         |  8 -----
 src/tests/default_executor_tests.cpp            | 34 +-------------------
 src/tests/health_check_tests.cpp                | 14 ++------
 src/tests/master_slave_reconciliation_tests.cpp |  8 -----
 src/tests/master_tests.cpp                      |  7 ----
 src/tests/mesos.cpp                             |  6 ++++
 src/tests/scheduler_tests.cpp                   |  6 ----
 15 files changed, 13 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 86cbba4..9246f42 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -5218,6 +5218,8 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
+  flags.authenticate_http_readwrite = true;
+
   Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> mesosContainerizer =
@@ -5740,6 +5742,8 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
+  flags.authenticate_http_readwrite = true;
+
   {
     mesos::ACL::AttachContainerInput* acl =
       flags.acls->add_attach_containers_input();

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/check_tests.cpp b/src/tests/check_tests.cpp
index 2a71f41..0163e3a 100644
--- a/src/tests/check_tests.cpp
+++ b/src/tests/check_tests.cpp
@@ -1335,12 +1335,6 @@ protected:
     slave::Flags flags = CheckTest::CreateSlaveFlags();
 
 #ifndef USE_SSL_SOCKET
-    // Disable operator API authentication for the default executor. Executor
-    // authentication currently has SSL as a dependency, so we cannot require
-    // executors to authenticate with the agent operator API if Mesos was not
-    // built with SSL support.
-    flags.authenticate_http_readwrite = false;
-
     // Set permissive ACLs in the agent so that the local authorizer will be
     // loaded and implicit executor authorization will be tested.
     ACLs acls;

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/container_daemon_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_daemon_tests.cpp b/src/tests/container_daemon_tests.cpp
index c615257..1396f25 100644
--- a/src/tests/container_daemon_tests.cpp
+++ b/src/tests/container_daemon_tests.cpp
@@ -89,8 +89,6 @@ TEST_F(ContainerDaemonTest, RestartOnTermination)
   ASSERT_SOME(jwtSecretKey);
 
   secretGenerator.reset(new JWTSecretGenerator(jwtSecretKey.get()));
-#else
-  slaveFlags.authenticate_http_readwrite = false;
 #endif // USE_SSL_SOCKET
 
   Future<Nothing> recover = FUTURE_DISPATCH(_, &Slave::__recover);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/containerizer/environment_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/environment_secret_isolator_tests.cpp b/src/tests/containerizer/environment_secret_isolator_tests.cpp
index fd1cd46..9728587 100644
--- a/src/tests/containerizer/environment_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/environment_secret_isolator_tests.cpp
@@ -139,14 +139,6 @@ TEST_F(EnvironmentSecretIsolatorTest, ResolveSecretDefaultExecutor)
 
   mesos::internal::slave::Flags flags = CreateSlaveFlags();
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Fetcher fetcher(flags);
   Try<SecretResolver*> secretResolver = SecretResolver::create();
   EXPECT_SOME(secretResolver);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_capabilities_isolator_tests.cpp b/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
index d3c4568..147f2cc 100644
--- a/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
@@ -344,14 +344,6 @@ TEST_P(LinuxCapabilitiesIsolatorTest, ROOT_NestedPing)
   flags.effective_capabilities = param.operator_effective;
   flags.bounding_capabilities = param.operator_bounding;
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   if (param.useImage == TestParam::WITH_IMAGE) {
     const string registry = path::join(sandbox.get(), "registry");
     AWAIT_READY(DockerArchive::create(registry, "test_image"));

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp b/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
index 9943178..d8c3e6d 100644
--- a/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
+++ b/src/tests/containerizer/nvidia_gpu_isolator_tests.cpp
@@ -680,14 +680,6 @@ TEST_F(NvidiaGpuTest, ROOT_CGROUPS_NVIDIA_GPU_DefaultExecutorVerifyDeviceAccess)
   flags.isolation = "filesystem/linux,cgroups/devices,gpu/nvidia";
   flags.resources = "cpus:1"; // To override the default with gpus:0.
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/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 3438030..0969cf5 100644
--- a/src/tests/containerizer/ports_isolator_tests.cpp
+++ b/src/tests/containerizer/ports_isolator_tests.cpp
@@ -1101,14 +1101,6 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_TaskGroup)
   flags.launcher = "linux";
   flags.check_agent_port_range_only = true;
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor.
-  // Executor authentication currently has SSL as a dependency, so we
-  // cannot require executors to authenticate with the agent operator
-  // API if Mesos was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
   ASSERT_SOME(slave);
@@ -1258,14 +1250,6 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverNestedBadTask)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor.
-  // Executor authentication currently has SSL as a dependency, so we
-  // cannot require executors to authenticate with the agent operator
-  // API if Mesos was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave =
     StartSlave(detector.get(), slaveId, flags);
@@ -1445,14 +1429,6 @@ TEST_F(NetworkPortsIsolatorTest, ROOT_NC_RecoverNestedGoodTask)
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor.
-  // Executor authentication currently has SSL as a dependency, so we
-  // cannot require executors to authenticate with the agent operator
-  // API if Mesos was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave =
     StartSlave(detector.get(), slaveId, flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/containerizer/posix_rlimits_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/posix_rlimits_isolator_tests.cpp b/src/tests/containerizer/posix_rlimits_isolator_tests.cpp
index f639cac..3af0044 100644
--- a/src/tests/containerizer/posix_rlimits_isolator_tests.cpp
+++ b/src/tests/containerizer/posix_rlimits_isolator_tests.cpp
@@ -363,14 +363,6 @@ TEST_F(PosixRLimitsIsolatorTest, NestedContainers)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "posix/rlimits";
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor.
-  // Executor authentication currently has SSL as a dependency, so we
-  // cannot require executors to authenticate with the agent operator
-  // API if Mesos was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/containerizer/volume_host_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_host_path_isolator_tests.cpp b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
index e4d0734..3c925bc 100644
--- a/src/tests/containerizer/volume_host_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
@@ -381,14 +381,6 @@ TEST_P(VolumeHostPathIsolatorMesosTest, ROOT_ChangeRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor.
-  // Executor authentication currently has SSL as a dependency, so we
-  // cannot require executors to authenticate with the agent operator
-  // API if Mesos was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index ee6dfbd..d2cf5fd 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -93,24 +93,7 @@ namespace tests {
 
 class DefaultExecutorTest
   : public MesosTest,
-    public WithParamInterface<string>
-{
-protected:
-  slave::Flags CreateSlaveFlags()
-  {
-    slave::Flags flags = MesosTest::CreateSlaveFlags();
-
-#ifndef USE_SSL_SOCKET
-    // Disable operator API authentication for the default executor. Executor
-    // authentication currently has SSL as a dependency, so we cannot require
-    // executors to authenticate with the agent operator API if Mesos was not
-    // built with SSL support.
-    flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
-    return flags;
-  }
-};
+    public WithParamInterface<string> {};
 
 
 // These tests are parameterized by the containerizers enabled on the agent.
@@ -2268,21 +2251,6 @@ public:
   PersistentVolumeDefaultExecutor() : param(GetParam()) {}
 
 protected:
-  slave::Flags CreateSlaveFlags()
-  {
-    slave::Flags flags = MesosTest::CreateSlaveFlags();
-
-#ifndef USE_SSL_SOCKET
-    // Disable operator API authentication for the default executor. Executor
-    // authentication currently has SSL as a dependency, so we cannot require
-    // executors to authenticate with the agent operator API if Mesos was not
-    // built with SSL support.
-    flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
-    return flags;
-  }
-
   LauncherAndIsolationParam param;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index bb909c5..1893c85 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -1941,13 +1941,8 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
 
+#ifndef USE_SSL_SOCKET
   // Set permissive ACLs in the agent so that the local authorizer will be
   // loaded and implicit executor authorization will be tested.
   ACLs acls;
@@ -2075,13 +2070,8 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
 
+#ifndef USE_SSL_SOCKET
   // Set permissive ACLs in the agent so that the local authorizer will be
   // loaded and implicit executor authorization will be tested.
   ACLs acls;

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/master_slave_reconciliation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_slave_reconciliation_tests.cpp b/src/tests/master_slave_reconciliation_tests.cpp
index 1a9b40a..26ff689 100644
--- a/src/tests/master_slave_reconciliation_tests.cpp
+++ b/src/tests/master_slave_reconciliation_tests.cpp
@@ -728,14 +728,6 @@ TEST_F(MasterSlaveReconciliationTest, SlaveReregisterTaskExecutorIds)
 
   slave::Flags flags = CreateSlaveFlags();
 
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
-
   StandaloneMasterDetector detector(master.get()->pid);
   Try<Owned<cluster::Slave>> slave = StartSlave(&detector, flags);
   ASSERT_SOME(slave);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index e41bd94..60718b4 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -8652,13 +8652,6 @@ TEST_P(MasterTestPrePostReservationRefinement, LaunchGroup)
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-#ifndef USE_SSL_SOCKET
-  // Disable operator API authentication for the default executor. Executor
-  // authentication currently has SSL as a dependency, so we cannot require
-  // executors to authenticate with the agent operator API if Mesos was not
-  // built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 1031283..d751b2e 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -228,6 +228,12 @@ slave::Flags MesosTest::CreateSlaveFlags()
 
     flags.jwt_secret_key = path;
   }
+#else // USE_SSL_SOCKET
+  // Disable operator API authentication for the default executor. Executor
+  // authentication currently has SSL as a dependency, so we cannot require
+  // executors to authenticate with the agent operator API if Mesos was not
+  // built with SSL support.
+  flags.authenticate_http_readwrite = false;
 #endif // USE_SSL_SOCKET
 
   {

http://git-wip-us.apache.org/repos/asf/mesos/blob/91ea75e8/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
index ecfce4d..87ef589 100644
--- a/src/tests/scheduler_tests.cpp
+++ b/src/tests/scheduler_tests.cpp
@@ -513,12 +513,6 @@ TEST_P(SchedulerTest, TaskGroupRunning)
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   slave::Flags flags = CreateSlaveFlags();
-#ifndef USE_SSL_SOCKET
-  // Executor authentication currently has SSL as a dependency, so we cannot
-  // require executors to authenticate with the agent operator API if Mesos
-  // was not built with SSL support.
-  flags.authenticate_http_readwrite = false;
-#endif // USE_SSL_SOCKET
 
   Owned<MasterDetector> detector = master.get()->createDetector();
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);