You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2017/04/13 16:26:17 UTC

[1/3] mesos git commit: Fixed docker fetcher 3xx redirect errors by header attached.

Repository: mesos
Updated Branches:
  refs/heads/master 578c9d000 -> f73f29bd3


Fixed docker fetcher 3xx redirect errors by header attached.

The root cause for this issue is that, in private registry
like quay.io, layer download request will be redirected to
storage server in S3. However, the curl command with '-L'
handles HTTP redirection automatically, in which case HTTP
headers will be attached to all requests. AmazonS3 server
will return 400 Bad Request if HTTP Authorization header
is attached, with 'InvalidArgument' error code. So we need
to touch the given URL first to add extra logic for HTTP
redirections.

Please note that the download() method is changed to be
recursive since no header should be attached once the
request get authenticated.

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


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

Branch: refs/heads/master
Commit: a2eaacb0cc2948dc08c498cb09b052ea7b0b3a6e
Parents: 578c9d0
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Apr 13 09:25:56 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Apr 13 09:25:56 2017 -0700

----------------------------------------------------------------------
 src/uri/fetchers/docker.cpp | 44 ++++++++++++++++++++++++++++++----------
 1 file changed, 33 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a2eaacb0/src/uri/fetchers/docker.cpp
----------------------------------------------------------------------
diff --git a/src/uri/fetchers/docker.cpp b/src/uri/fetchers/docker.cpp
index d6d2e8e..1c6ab92 100644
--- a/src/uri/fetchers/docker.cpp
+++ b/src/uri/fetchers/docker.cpp
@@ -206,19 +206,16 @@ static Future<http::Response> curl(
 
 // TODO(jieyu): Add a comment here.
 static Future<int> download(
-    const URI& uri,
-    const string& directory,
+    const string& uri,
+    const string& blobPath,
     const http::Headers& headers = http::Headers())
 {
-  const string output = path::join(directory, Path(uri.path()).basename());
-
   vector<string> argv = {
     "curl",
     "-s",                 // Don't show progress meter or error messages.
     "-S",                 // Make curl show an error message if it fails.
-    "-L",                 // Follow HTTP 3xx redirects.
-    "-w", "%{http_code}", // Display HTTP response code on stdout.
-    "-o", output          // Write output to the file.
+    "-w", "%{http_code}\n%{redirect_url}", // Display HTTP response code and the redirected URL. // NOLINT(whitespace/line_length)
+    "-o", blobPath        // Write output to the file.
   };
 
   // Add additional headers.
@@ -227,7 +224,7 @@ static Future<int> download(
     argv.push_back(key + ": " + value);
   }
 
-  argv.push_back(strings::trim(stringify(uri)));
+  argv.push_back(uri);
 
   // TODO(jieyu): Kill the process if discard is called.
   Try<Subprocess> s = subprocess(
@@ -245,7 +242,7 @@ static Future<int> download(
       s.get().status(),
       io::read(s.get().out().get()),
       io::read(s.get().err().get()))
-    .then([](const tuple<
+    .then([=](const tuple<
         Future<Option<int>>,
         Future<string>,
         Future<string>>& t) -> Future<int> {
@@ -278,16 +275,41 @@ static Future<int> download(
             (output.isFailed() ? output.failure() : "discarded"));
       }
 
+      vector<string> tokens = strings::tokenize(output.get(), "\n", 2);
+      if (tokens.empty()) {
+        return Failure("Unexpected 'curl' output: " + output.get());
+      }
+
       // Parse the output and get the HTTP response code.
-      Try<int> code = numify<int>(output.get());
+      Try<int> code = numify<int>(tokens[0]);
       if (code.isError()) {
-        return Failure("Unexpected output from 'curl': " + output.get());
+        return Failure(
+            "Unexpected HTTP response code from 'curl': " + tokens[0]);
+      }
+
+      // If there are two tokens, it means that the redirect url
+      // exists in the stdout and the request to download the blob
+      // is already authenticated.
+      if (tokens.size() == 2) {
+        // Headers are not attached because the request is already
+        // authenticated.
+        return download(tokens[1], blobPath);
       }
 
       return code.get();
     });
 }
 
+
+static Future<int> download(
+    const URI& uri,
+    const string& directory,
+    const http::Headers& headers = http::Headers())
+{
+  const string blobPath = path::join(directory, Path(uri.path()).basename());
+  return download(strings::trim(stringify(uri)), blobPath, headers);
+}
+
 //-------------------------------------------------------------------
 // DockerFetcherPlugin implementation.
 //-------------------------------------------------------------------


[2/3] mesos git commit: Parameterized the existing alpine based test with more registries.

Posted by ji...@apache.org.
Parameterized the existing alpine based test with more registries.

Now we test alpine image from three registries:
1. docker hub.
2. quay.io.
3. alicloud.

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


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

Branch: refs/heads/master
Commit: d1015ffe35dfcd353d7dbbd187244586feddd6a7
Parents: a2eaacb
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Apr 13 09:26:00 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Apr 13 09:26:00 2017 -0700

----------------------------------------------------------------------
 .../containerizer/provisioner_docker_tests.cpp  | 21 +++++++++++++++++---
 1 file changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d1015ffe/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index dbbc353..4dbd911 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -368,7 +368,9 @@ TEST_F(ProvisionerDockerLocalStoreTest, PullingSameImageSimutanuously)
 
 
 #ifdef __linux__
-class ProvisionerDockerPullerTest : public MesosTest {};
+class ProvisionerDockerPullerTest
+  : public MesosTest,
+    public WithParamInterface<string> {};
 
 
 // This test verifies that local docker image can be pulled and
@@ -448,9 +450,22 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_LocalPullerSimpleCommand)
 }
 
 
+// For official Docker images, users can omit the 'library/' prefix
+// when specifying the repository name (e.g., 'busybox'). The registry
+// puller normalize docker official images if necessary.
+INSTANTIATE_TEST_CASE_P(
+    ImageAlpine,
+    ProvisionerDockerPullerTest,
+    ::testing::ValuesIn(vector<string>({
+        "alpine", // Verifies the normalization of the Docker repository name.
+        "library/alpine",
+        "quay.io/coreos/alpine-sh",
+        "registry.cn-hangzhou.aliyuncs.com/acs-sample/alpine"})));
+
+
 // TODO(jieyu): This is a ROOT test because of MESOS-4757. Remove the
 // ROOT restriction after MESOS-4757 is resolved.
-TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
+TEST_P(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -498,7 +513,7 @@ TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
 
   Image image;
   image.set_type(Image::DOCKER);
-  image.mutable_docker()->set_name("library/alpine");
+  image.mutable_docker()->set_name(GetParam());
 
   ContainerInfo* container = task.mutable_container();
   container->set_type(ContainerInfo::MESOS);


[3/3] mesos git commit: Removed a duplicate unit test 'ROOT_INTERNET_CURL_Normalize'.

Posted by ji...@apache.org.
Removed a duplicate unit test 'ROOT_INTERNET_CURL_Normalize'.

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


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

Branch: refs/heads/master
Commit: f73f29bd3f1399847309fa801f6c3c0a022d57fc
Parents: d1015ff
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Apr 13 09:26:08 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Apr 13 09:26:08 2017 -0700

----------------------------------------------------------------------
 .../containerizer/provisioner_docker_tests.cpp  | 75 --------------------
 1 file changed, 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f73f29bd/src/tests/containerizer/provisioner_docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_docker_tests.cpp b/src/tests/containerizer/provisioner_docker_tests.cpp
index 4dbd911..b0a4d21 100644
--- a/src/tests/containerizer/provisioner_docker_tests.cpp
+++ b/src/tests/containerizer/provisioner_docker_tests.cpp
@@ -540,81 +540,6 @@ TEST_P(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_SimpleCommand)
 }
 
 
-// This test verifies the normalization of the Docker repository name.
-// For official Docker images, users can omit the 'library/' prefix
-// when specifying the repository name (e.g., 'busybox').
-TEST_F(ProvisionerDockerPullerTest, ROOT_INTERNET_CURL_Normalize)
-{
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = "docker/runtime,filesystem/linux";
-  flags.image_providers = "docker";
-
-  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];
-
-  CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/ls");
-  command.add_arguments("ls");
-  command.add_arguments("-al");
-  command.add_arguments("/");
-
-  TaskInfo task = createTask(
-      offer.slave_id(),
-      Resources::parse("cpus:1;mem:128").get(),
-      command);
-
-  Image image;
-  image.set_type(Image::DOCKER);
-  image.mutable_docker()->set_name("alpine");
-
-  ContainerInfo* container = task.mutable_container();
-  container->set_type(ContainerInfo::MESOS);
-  container->mutable_mesos()->mutable_image()->CopyFrom(image);
-
-  Future<TaskStatus> statusRunning;
-  Future<TaskStatus> statusFinished;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&statusRunning))
-    .WillOnce(FutureArg<1>(&statusFinished));
-
-  driver.launchTasks(offer.id(), {task});
-
-  AWAIT_READY_FOR(statusRunning, Seconds(60));
-  EXPECT_EQ(task.task_id(), statusRunning->task_id());
-  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
-
-  AWAIT_READY(statusFinished);
-  EXPECT_EQ(task.task_id(), statusFinished->task_id());
-  EXPECT_EQ(TASK_FINISHED, statusFinished->state());
-
-  driver.stop();
-  driver.join();
-}
-
-
 // This test verifies that the scratch based docker image (that
 // only contain a single binary and its dependencies) can be
 // launched correctly.