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/02/02 18:43:49 UTC

[1/3] mesos git commit: Implemented new http::Headers abstraction for WWW-Authenticate.

Repository: mesos
Updated Branches:
  refs/heads/master f179400b1 -> 274cc3354


Implemented new http::Headers abstraction for WWW-Authenticate.

This patch introduced a new abstraction for http::Headers and helper
methods for the http::Headers hashmap, as well as a new
http::WWWAuthenticate class for the new `Headers` abstraction.

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


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

Branch: refs/heads/master
Commit: 95c177e95a529248718e3bbd6350f9c68ba68220
Parents: f179400
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Feb 2 10:16:05 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Feb 2 10:16:05 2017 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/include/process/http.hpp |  97 ++++++++++++++++--
 3rdparty/libprocess/src/http.cpp             | 117 ++++++++++++++++++++++
 2 files changed, 208 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/95c177e9/3rdparty/libprocess/include/process/http.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/http.hpp b/3rdparty/libprocess/include/process/http.hpp
index 4b11e08..3a40696 100644
--- a/3rdparty/libprocess/include/process/http.hpp
+++ b/3rdparty/libprocess/include/process/http.hpp
@@ -256,12 +256,6 @@ struct Status
 };
 
 
-typedef hashmap<std::string,
-                std::string,
-                CaseInsensitiveHash,
-                CaseInsensitiveEqual> Headers;
-
-
 // Represents an asynchronous in-memory unbuffered Pipe, currently
 // used for streaming HTTP responses via chunked encoding. Note that
 // being an in-memory pipe means that this cannot be used across OS
@@ -422,6 +416,97 @@ private:
 };
 
 
+namespace header {
+
+// https://tools.ietf.org/html/rfc2617.
+class WWWAuthenticate
+{
+public:
+  static constexpr const char* NAME = "WWW-Authenticate";
+
+  WWWAuthenticate(
+      const std::string& authScheme,
+      const hashmap<std::string, std::string>& authParam)
+    : authScheme_(authScheme),
+      authParam_(authParam) {}
+
+  static Try<WWWAuthenticate> create(const std::string& value);
+
+  std::string authScheme();
+  hashmap<std::string, std::string> authParam();
+
+private:
+  // According to RFC, HTTP/1.1 server may return multiple challenges
+  // with a 401 (Authenticate) response. Each challenage is in the
+  // format of 'auth-scheme 1*SP 1#auth-param' and each challenage may
+  // use a different auth-scheme.
+  // https://tools.ietf.org/html/rfc2617#section-4.6
+  //
+  // TODO(gilbert): We assume there is only one authenticate challenge.
+  // Multiple challenges should be supported as well.
+  std::string authScheme_;
+  hashmap<std::string, std::string> authParam_;
+};
+
+} // namespace header {
+
+
+class Headers
+{
+public:
+  typedef hashmap<
+      std::string,
+      std::string,
+      CaseInsensitiveHash,
+      CaseInsensitiveEqual> Type;
+
+  Headers() {}
+  Headers(const Type& _headers) : headers(_headers) {}
+
+  template <typename T>
+  Result<T> get() const
+  {
+    Option<std::string> value = get(T::NAME);
+    if (value.isNone()) {
+      return None();
+    }
+    Try<T> header = T::create(value.get());
+    if (header.isError()) {
+      return Error(header.error());
+    }
+    return header.get();
+  }
+
+  Headers& operator=(const Type& _headers);
+
+  std::string& operator[](const std::string& key);
+
+  void put(const std::string& key, const std::string& value);
+
+  Option<std::string> get(const std::string& key) const;
+
+  std::string& at(const std::string& key);
+
+  const std::string& at(const std::string& key) const;
+
+  bool contains(const std::string& key) const;
+
+  size_t size() const;
+
+  bool empty() const;
+
+  void clear();
+
+  typename Type::iterator begin() { return headers.begin(); }
+  typename Type::iterator end() { return headers.end(); }
+  typename Type::const_iterator begin() const { return headers.cbegin(); }
+  typename Type::const_iterator end() const { return headers.cend(); }
+
+private:
+  Type headers;
+};
+
+
 struct Request
 {
   Request()

http://git-wip-us.apache.org/repos/asf/mesos/blob/95c177e9/3rdparty/libprocess/src/http.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/http.cpp b/3rdparty/libprocess/src/http.cpp
index 4fd62c8..439840a 100644
--- a/3rdparty/libprocess/src/http.cpp
+++ b/3rdparty/libprocess/src/http.cpp
@@ -597,6 +597,123 @@ Future<Nothing> Pipe::Writer::readerClosed() const
 }
 
 
+namespace header {
+
+Try<WWWAuthenticate> WWWAuthenticate::create(const string& value)
+{
+  // Set `maxTokens` as 2 since auth-param quoted string may
+  // contain space (e.g., "Basic realm="Registry Realm").
+  vector<string> tokens = strings::tokenize(value, " ", 2);
+  if (tokens.size() != 2) {
+    return Error("Unexpected WWW-Authenticate header format: '" + value + "'");
+  }
+
+  hashmap<string, string> authParam;
+  foreach (const string& token, strings::split(tokens[1], ",")) {
+    vector<string> split = strings::split(token, "=");
+    if (split.size() != 2) {
+      return Error(
+          "Unexpected auth-param format: '" +
+          token + "' in '" + tokens[1] + "'");
+    }
+
+    // Auth-param values can be a quoted-string or directive values.
+    // Please see section "3.2.2.4 Directive values and quoted-string":
+    // https://tools.ietf.org/html/rfc2617.
+    authParam[split[0]] = strings::trim(split[1], strings::ANY, "\"");
+  }
+
+  // The realm directive (case-insensitive) is required for all
+  // authentication schemes that issue a challenge.
+  if (!authParam.contains("realm")) {
+    return Error(
+        "Unexpected auth-param '" +
+        tokens[1] + "': 'realm' is not defined");
+  }
+
+  return WWWAuthenticate(tokens[0], authParam);
+}
+
+
+string WWWAuthenticate::authScheme()
+{
+  return authScheme_;
+}
+
+
+hashmap<string, string> WWWAuthenticate::authParam()
+{
+  return authParam_;
+}
+
+} // namespace header {
+
+
+Headers& Headers::operator=(const Headers::Type& _headers)
+{
+  headers = _headers;
+  return *this;
+}
+
+
+string& Headers::operator[](const string& key)
+{
+  return headers[key];
+}
+
+
+void Headers::put(const string& key, const string& value)
+{
+  headers.put(key, value);
+}
+
+
+Option<string> Headers::get(const string& key) const
+{
+  if (headers.contains(key)) {
+    return headers.at(key);
+  }
+
+  return None();
+}
+
+
+string& Headers::at(const string& key)
+{
+  return headers.at(key);
+}
+
+
+const string& Headers::at(const string& key) const
+{
+  return headers.at(key);
+}
+
+
+bool Headers::contains(const string& key) const
+{
+  return headers.contains(key);
+}
+
+
+size_t Headers::size() const
+{
+  return headers.size();
+}
+
+
+bool Headers::empty() const
+{
+  return headers.empty();
+}
+
+
+void Headers::clear()
+{
+  headers.clear();
+}
+
+
 OK::OK(const JSON::Value& value, const Option<string>& jsonp)
   : Response(Status::OK)
 {


[3/3] mesos git commit: Support 'Basic' auth docker registry on Unified Containerizer.

Posted by ji...@apache.org.
Support 'Basic' auth docker registry on Unified Containerizer.

This patch implements the support for 'Basic' docker registry
authorization. It is tested by a local authenticated private
registry using 'localhost:443/alpine' docker image.
Please note that the AWS ECS uses Basic authorization but it
does not work yet due to the redirect issue MESOS-5172.

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


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

Branch: refs/heads/master
Commit: 274cc3354c6999e2efee46a9e2c9d5b3069eed64
Parents: 988b520
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Feb 2 10:16:16 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Feb 2 10:16:16 2017 -0800

----------------------------------------------------------------------
 src/uri/fetchers/docker.cpp | 330 +++++++++++++++++++--------------------
 1 file changed, 165 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/274cc335/src/uri/fetchers/docker.cpp
----------------------------------------------------------------------
diff --git a/src/uri/fetchers/docker.cpp b/src/uri/fetchers/docker.cpp
index 5dd7b91..68f380d 100644
--- a/src/uri/fetchers/docker.cpp
+++ b/src/uri/fetchers/docker.cpp
@@ -15,7 +15,6 @@
 // limitations under the License.
 
 #include <list>
-#include <sstream>
 #include <string>
 #include <tuple>
 #include <vector>
@@ -49,7 +48,6 @@ namespace io = process::io;
 namespace spec = docker::spec;
 
 using std::list;
-using std::ostringstream;
 using std::set;
 using std::string;
 using std::tuple;
@@ -313,22 +311,22 @@ private:
   Future<Nothing> __fetch(
       const URI& uri,
       const string& directory,
-      const Option<string>& authToken,
+      const http::Headers& authHeaders,
       const http::Response& response);
 
   Future<Nothing> fetchBlob(
       const URI& uri,
       const string& directory,
-      const Option<string>& authToken);
+      const http::Headers& authHeaders);
 
   Future<Nothing> _fetchBlob(
       const URI& uri,
       const string& directory,
       const URI& blobUri);
 
-  Future<string> getAuthToken(const http::Response& response, const URI& uri);
-  http::Headers getAuthHeaderBasic(const Option<string>& credential);
-  http::Headers getAuthHeaderBearer(const Option<string>& authToken);
+  Future<http::Headers> getAuthHeader(
+      const URI& uri,
+      const http::Response& response);
 
   URI getManifestUri(const URI& uri);
   URI getBlobUri(const URI& uri);
@@ -418,6 +416,9 @@ Future<Nothing> DockerFetcherPluginProcess::fetch(
     const URI& uri,
     const string& directory)
 {
+  // TODO(gilbert): Convert the `uri` to ::docker::spec::ImageReference
+  // and pass it all the way down to avoid the complicated URI conversion
+  // and make the code more readable.
   if (schemes().count(uri.scheme()) == 0) {
     return Failure(
         "Docker fetcher plugin does not support "
@@ -440,7 +441,7 @@ Future<Nothing> DockerFetcherPluginProcess::fetch(
   }
 
   if (uri.scheme() == "docker-blob") {
-    return fetchBlob(uri, directory, None());
+    return fetchBlob(uri, directory, http::Headers());
   }
 
   URI manifestUri = getManifestUri(uri);
@@ -462,26 +463,27 @@ Future<Nothing> DockerFetcherPluginProcess::_fetch(
     const http::Response& response)
 {
   if (response.code == http::Status::UNAUTHORIZED) {
-    return getAuthToken(response, manifestUri)
-      .then(defer(self(), [=](const string& authToken) -> Future<Nothing> {
-        return curl(manifestUri, getAuthHeaderBearer(authToken))
+    return getAuthHeader(manifestUri, response)
+      .then(defer(self(), [=](
+          const http::Headers& authHeaders) -> Future<Nothing> {
+        return curl(manifestUri, authHeaders)
           .then(defer(self(),
                       &Self::__fetch,
                       uri,
                       directory,
-                      authToken,
+                      authHeaders,
                       lambda::_1));
       }));
   }
 
-  return __fetch(uri, directory, None(), response);
+  return __fetch(uri, directory, http::Headers(), response);
 }
 
 
 Future<Nothing> DockerFetcherPluginProcess::__fetch(
     const URI& uri,
     const string& directory,
-    const Option<string>& authToken,
+    const http::Headers& authHeaders,
     const http::Response& response)
 {
   if (response.code != http::Status::OK) {
@@ -543,7 +545,7 @@ Future<Nothing> DockerFetcherPluginProcess::__fetch(
     futures.push_back(fetchBlob(
         blob,
         directory,
-        authToken));
+        authHeaders));
   }
 
   return collect(futures)
@@ -554,20 +556,20 @@ Future<Nothing> DockerFetcherPluginProcess::__fetch(
 Future<Nothing> DockerFetcherPluginProcess::fetchBlob(
     const URI& uri,
     const string& directory,
-    const Option<string>& authToken)
+    const http::Headers& authHeaders)
 {
   URI blobUri = getBlobUri(uri);
 
-  return download(blobUri, directory, getAuthHeaderBearer(authToken))
+  return download(blobUri, directory, authHeaders)
     .then(defer(self(), [=](int code) -> Future<Nothing> {
       if (code == http::Status::OK) {
         return Nothing();
       }
 
-      // Note that if 'authToken' is specified, but we still get a
+      // Note that if 'authHeaders' is not empty, but we still get a
       // '401 Unauthorized' response, we return a Failure. This can
       // prevent us from entering an infinite loop.
-      if (code == http::Status::UNAUTHORIZED && authToken.isNone()) {
+      if (code == http::Status::UNAUTHORIZED && authHeaders.empty()) {
         return _fetchBlob(uri, directory, blobUri);
       }
 
@@ -597,7 +599,7 @@ Future<Nothing> DockerFetcherPluginProcess::_fetchBlob(
           "but get '" + response.status + "' instead");
       }
 
-      return getAuthToken(response, blobUri)
+      return getAuthHeader(blobUri, response)
         .then(defer(self(),
                     &Self::fetchBlob,
                     uri,
@@ -607,186 +609,184 @@ Future<Nothing> DockerFetcherPluginProcess::_fetchBlob(
 }
 
 
-// If a '401 Unauthorized' response is received, we expect a header
-// 'Www-Authenticate' containing the auth server information. This
-// function takes the '401 Unauthorized' response, extracts the auth
-// server information, and then contacts the auth server to get the
-// token. The token will then be placed in the subsequent HTTP
-// requests as a header.
-//
-// See details here:
-// https://docs.docker.com/registry/spec/auth/token/
-Future<string> DockerFetcherPluginProcess::getAuthToken(
-    const http::Response& response,
-    const URI& uri)
+static http::Headers getAuthHeaderBasic(
+    const Option<string>& credential)
 {
-  // The expected HTTP response here is:
-  //
-  // HTTP/1.1 401 Unauthorized
-  // Www-Authenticate: Bearer realm="xxx",service="yyy",scope="zzz"
-  CHECK_EQ(response.code, http::Status::UNAUTHORIZED);
-
-  if (!response.headers.contains("WWW-Authenticate")) {
-    return Failure("WWW-Authorization header is not found");
-  }
-
-  const vector<string> tokens = strings::tokenize(
-      response.headers.at("WWW-Authenticate"), " ");
+  http::Headers headers;
 
-  if (tokens.size() != 2) {
-    return Failure(
-        "Unexpected WWW-Authenticate header format: "
-        "'" + response.headers.at("WWW-Authenticate") + "'");
+  if (credential.isSome()) {
+    // NOTE: The 'Basic' credential would be attached as a header
+    // when pulling a public image from a registry, if the host
+    // of the image's repository exists in the docker config file.
+    headers["Authorization"] = "Basic " + credential.get();
   }
 
-  if (tokens[0] != "Bearer") {
-    return Failure("Not a Bearer authentication challenge");
-  }
+  return headers;
+}
 
-  // Map containing the 'realm', 'service' and 'scope' information.
-  hashmap<string, string> attributes;
 
-  foreach (const string& token, strings::tokenize(tokens[1], ",")) {
-    const vector<string> split = strings::split(token, "=");
-    if (split.size() != 2) {
-      return Failure("Unexpected attribute format: '" + token + "'");
-    }
+static http::Headers getAuthHeaderBearer(
+    const Option<string>& authToken)
+{
+  http::Headers headers;
 
-    attributes[split[0]] = strings::trim(split[1], strings::ANY, "\"");
+  if (authToken.isSome()) {
+    headers["Authorization"] = "Bearer " + authToken.get();
   }
 
-  if (!attributes.contains("realm")) {
-    return Failure("Missing 'realm' in WWW-Authenticate header");
-  }
+  return headers;
+}
 
-  if (!attributes.contains("service")) {
-    return Failure("Missing 'service' in WWW-Authenticate header");
-  }
 
-  if (!attributes.contains("scope")) {
-    return Failure("Missing 'scope' in WWW-Authenticate header");
+Future<http::Headers> DockerFetcherPluginProcess::getAuthHeader(
+    const URI& uri,
+    const http::Response& response)
+{
+  Result<http::header::WWWAuthenticate> header =
+    response.headers.get<http::header::WWWAuthenticate>();
+
+  if (header.isError()) {
+    return Failure(
+        "Failed to get WWW-Authenticate header: " + header.error());
+  } else if (header.isNone()) {
+    return Failure("Unexpected empty WWW-Authenticate header");
   }
 
-  ostringstream stream;
-
-  // TODO(jieyu): Currently, we don't expect the auth server to return
-  // a service or a scope that needs encoding.
-  string authServerUri =
-    attributes.at("realm") + "?" +
-    "service=" + attributes.at("service") + "&" +
-    "scope=" + attributes.at("scope");
-
-  Option<string> auth;
-
-  // TODO(gilbert): Ideally, this should be done after getting
-  // the '401 Unauthorized' response. Then, the workflow should
-  // be:
-  // 1. Send a requst to registry for pulling.
-  // 2. The registry returns '401 Unauthorized' HTTP response.
-  // 3. The registry client makes a request (without a Basic header)
-  //    to the authorization server for a Bearer token.
-  // 4. The authorization servicer returns an unacceptable
-  //    Bearer token.
-  // 5. Re-send a request to registry with the Bearer token attached.
-  // 6. The registry returns '401 Unauthorized' HTTP response.
-  // 7. The registry client makes a request (with a correct Basic
-  //    header attached) to the authorization server for a Bearer
-  //    token.
-  // 8. The authorization servicer returns a corrent Bearer token.
-  // 9. Re-send a request to registry with the right Bearer token
-  //    attached.
-  // 10. The registry authorizes the client, and the docker fetcher
-  //     starts pulling.
-  // The step 3 ~ 6 are exactly what this TODO describes.
-
-  // TODO(gilbert): Currrently, the docker fetcher plugin only
-  // supports Basic Authentication. From Docker 1.11, the docker
-  // engine supports both Basic authentication and OAuth2 for
-  // getting tokens. Ideally, we should support both in docker
-  // fetcher plugin.
-  foreachpair (const string& key, const spec::Config::Auth& value, auths) {
-    // Handle domains including 'docker.io' as a special case,
-    // because the url is set differently for different version
-    // of docker default registry, but all of them should depend
-    // on the same default namespace 'docker.io'. Please see:
-    // https://github.com/docker/docker/blob/master/registry/config.go#L34
-    const bool isDocker =
-      strings::contains(uri.host(), "docker.io") &&
-      strings::contains(key, "docker.io");
+  // According to RFC, auth scheme should be case insensitive.
+  const string authScheme = strings::upper(header->authScheme());
 
-    // NOTE: The host field of uri can be either domain or IP
-    // address, which is merged in docker registry puller.
+  // If a '401 Unauthorized' response is received and the auth-scheme
+  // is 'Basic', we do basic authentication with the server directly.
+  if (authScheme == "BASIC") {
     const string registry = uri.has_port()
       ? uri.host() + ":" + stringify(uri.port())
       : uri.host();
 
-    // Should not use 'http::URL::parse()' here, since many
-    // registry domain recorded in docker config file does
-    // not start with 'https://' or 'http://'. They are pure
-    // domain only (e.g., 'quay.io', 'localhost:5000').
-    // Please see 'ResolveAuthConfig()' in:
-    // https://github.com/docker/docker/blob/master/registry/auth.go
-    if (isDocker || (registry == spec::parseAuthUrl(key))) {
-      if (value.has_auth()) {
+    Option<string> auth;
+    foreachpair (const string& key, const spec::Config::Auth& value, auths) {
+      if (registry == spec::parseAuthUrl(key) && value.has_auth()) {
         auth = value.auth();
         break;
       }
     }
-  }
-
-  return curl(authServerUri, getAuthHeaderBasic(auth))
-    .then([authServerUri](const http::Response& response) -> Future<string> {
-      if (response.code != http::Status::OK) {
-        return Failure(
-          "Unexpected HTTP response '" + response.status + "' "
-          "when trying to GET '" + authServerUri + "'");
-      }
 
-      CHECK_EQ(response.type, http::Response::BODY);
+    return getAuthHeaderBasic(auth);
+  }
 
-      Try<JSON::Object> object = JSON::parse<JSON::Object>(response.body);
-      if (object.isError()) {
-        return Failure("Parsing the JSON object failed: " + object.error());
-      }
+  // If a '401 Unauthorized' response is received and the auth-scheme
+  // is 'Bearer', we expect a header 'Www-Authenticate' containing the
+  // auth server information. We extract the auth server information
+  // from the auth-param, and then contacts the auth server to get the
+  // token. The token will then be placed in the subsequent HTTP
+  // requests as a header.
+  //
+  // See details here:
+  // https://docs.docker.com/registry/spec/auth/token/
+  if (authScheme == "BEARER") {
+    hashmap<string, string> authParam = header->authParam();
+
+    // `authParam` is supposed to contain the 'realm', 'service'
+    // and 'scope' information for bearer authentication.
+    if (!authParam.contains("realm")) {
+      return Failure("Missing 'realm' in WWW-Authenticate header");
+    }
 
-      Result<JSON::String> token = object->find<JSON::String>("token");
-      if (token.isError()) {
-        return Failure("Finding token in JSON object failed: " + token.error());
-      } else if (token.isNone()) {
-        return Failure("Failed to find token in JSON object");
-      }
+    if (!authParam.contains("service")) {
+      return Failure("Missing 'service' in WWW-Authenticate header");
+    }
 
-      return token->value;
-    });
-}
+    if (!authParam.contains("scope")) {
+      return Failure("Missing 'scope' in WWW-Authenticate header");
+    }
 
+    // NOTE: The host field of uri can be either domain or IP
+    // address, which is merged in docker registry puller.
+    const string registry = uri.has_port()
+      ? uri.host() + ":" + stringify(uri.port())
+      : uri.host();
 
-http::Headers DockerFetcherPluginProcess::getAuthHeaderBasic(
-    const Option<string>& credential)
-{
-  http::Headers headers;
+    // TODO(gilbert): Ideally, this should be done after getting
+    // the '401 Unauthorized' response. Then, the workflow should
+    // be:
+    // 1. Send a requst to registry for pulling.
+    // 2. The registry returns '401 Unauthorized' HTTP response.
+    // 3. The registry client makes a request (without a Basic header)
+    //    to the authorization server for a Bearer token.
+    // 4. The authorization servicer returns an unacceptable
+    //    Bearer token.
+    // 5. Re-send a request to registry with the Bearer token attached.
+    // 6. The registry returns '401 Unauthorized' HTTP response.
+    // 7. The registry client makes a request (with a correct Basic
+    //    header attached) to the authorization server for a Bearer
+    //    token.
+    // 8. The authorization servicer returns a corrent Bearer token.
+    // 9. Re-send a request to registry with the right Bearer token
+    //    attached.
+    // 10. The registry authorizes the client, and the docker fetcher
+    //     starts pulling.
+    // The step 3 ~ 6 are exactly what this TODO describes.
+    Option<string> auth;
+
+    foreachpair (const string& key, const spec::Config::Auth& value, auths) {
+      // Handle domains including 'docker.io' as a special case,
+      // because the url is set differently for different version
+      // of docker default registry, but all of them should depend
+      // on the same default namespace 'docker.io'. Please see:
+      // https://github.com/docker/docker/blob/master/registry/config.go#L34
+      const bool isDocker =
+        strings::contains(uri.host(), "docker.io") &&
+        strings::contains(key, "docker.io");
+
+      // Should not use 'http::URL::parse()' here, since many
+      // registry domain recorded in docker config file does
+      // not start with 'https://' or 'http://'. They are pure
+      // domain only (e.g., 'quay.io', 'localhost:5000').
+      // Please see 'ResolveAuthConfig()' in:
+      // https://github.com/docker/docker/blob/master/registry/auth.go
+      if (isDocker || (registry == spec::parseAuthUrl(key))) {
+        if (value.has_auth()) {
+          auth = value.auth();
+          break;
+        }
+      }
+    }
 
-  if (credential.isSome()) {
-    // NOTE: The 'Basic' credential would be attached as a header
-    // when pulling a public image from a registry, if the host
-    // of the image's repository exists in the docker config file.
-    headers["Authorization"] = "Basic " + credential.get();
-  }
+    // TODO(jieyu): Currently, we don't expect the auth server to return
+    // a service or a scope that needs encoding.
+    string authServerUri =
+      authParam.at("realm") + "?" +
+      "service=" + authParam.at("service") + "&" +
+      "scope=" + authParam.at("scope");
+
+    return curl(authServerUri, getAuthHeaderBasic(auth))
+      .then([authServerUri](
+          const http::Response& response) -> Future<http::Headers> {
+        if (response.code != http::Status::OK) {
+          return Failure(
+            "Unexpected HTTP response '" + response.status + "' "
+            "when trying to GET '" + authServerUri + "'");
+        }
 
-  return headers;
-}
+        CHECK_EQ(response.type, http::Response::BODY);
 
+        Try<JSON::Object> object = JSON::parse<JSON::Object>(response.body);
+        if (object.isError()) {
+          return Failure("Parsing the JSON object failed: " + object.error());
+        }
 
-http::Headers DockerFetcherPluginProcess::getAuthHeaderBearer(
-    const Option<string>& authToken)
-{
-  http::Headers headers;
+        Result<JSON::String> token = object->find<JSON::String>("token");
+        if (token.isError()) {
+          return Failure(
+              "Finding token in JSON object failed: " + token.error());
+        } else if (token.isNone()) {
+          return Failure("Failed to find token in JSON object");
+        }
 
-  if (authToken.isSome()) {
-    headers["Authorization"] = "Bearer " + authToken.get();
+        return getAuthHeaderBearer(token->value);
+      });
   }
 
-  return headers;
+  return Failure("Unsupported auth-scheme: " + authScheme);
 }
 
 


[2/3] mesos git commit: Fixed mesos test helper createBasicAuthHeaders() on http header.

Posted by ji...@apache.org.
Fixed mesos test helper createBasicAuthHeaders() on http header.

Fixed the initialization of http::Headers in mesos test helper.

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


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

Branch: refs/heads/master
Commit: 988b52039e8e052e0722deafc26c972c762e89ac
Parents: 95c177e
Author: Gilbert Song <so...@gmail.com>
Authored: Thu Feb 2 10:16:09 2017 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Feb 2 10:16:09 2017 -0800

----------------------------------------------------------------------
 src/tests/mesos.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/988b5203/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 71e9197..ff83a9c 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -919,11 +919,11 @@ template <typename TCredential>
 inline process::http::Headers createBasicAuthHeaders(
     const TCredential& credential)
 {
-  return process::http::Headers{{
+  return process::http::Headers({{
       "Authorization",
       "Basic " +
         base64::encode(credential.principal() + ":" + credential.secret())
-  }};
+  }});
 }