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 2016/03/03 03:06:21 UTC

[1/3] mesos git commit: Introduced copy fetcher plugin for file URIs.

Repository: mesos
Updated Branches:
  refs/heads/master 63e4f4e22 -> f0fb40461


Introduced copy fetcher plugin for file URIs.

This change adds support for fetching `file` URIs using the fetcher
plugin framework.

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


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

Branch: refs/heads/master
Commit: fc113e68fb60e1f7e384219c193d3c1416bc23e3
Parents: 63e4f4e
Author: Jojy Varghese <jo...@mesosphere.io>
Authored: Wed Mar 2 18:03:56 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Mar 2 18:03:56 2016 -0800

----------------------------------------------------------------------
 src/CMakeLists.txt              |   1 +
 src/Makefile.am                 |   2 +
 src/tests/uri_fetcher_tests.cpp |  43 +++++++++++
 src/uri/fetcher.cpp             |   1 +
 src/uri/fetcher.hpp             |   2 +
 src/uri/fetchers/copy.cpp       | 134 +++++++++++++++++++++++++++++++++++
 src/uri/fetchers/copy.hpp       |  52 ++++++++++++++
 7 files changed, 235 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 9d29fad..0eabfad 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -292,6 +292,7 @@ endif (NOT WIN32)
 set(URI_SRC
   uri/fetcher.cpp
   uri/utils.cpp
+  uri/fetchers/copy.cpp
   uri/fetchers/curl.cpp
   uri/fetchers/docker.cpp
   uri/fetchers/hadoop.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index fe549a8..b30cc25 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -681,6 +681,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/resource_estimators/noop.cpp					\
   uri/fetcher.cpp							\
   uri/utils.cpp								\
+  uri/fetchers/copy.cpp							\
   uri/fetchers/curl.cpp							\
   uri/fetchers/docker.cpp						\
   uri/fetchers/hadoop.cpp						\
@@ -809,6 +810,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   tests/containerizer/store.hpp						\
   uri/fetcher.hpp							\
   uri/utils.hpp								\
+  uri/fetchers/copy.hpp							\
   uri/fetchers/curl.hpp							\
   uri/fetchers/docker.hpp						\
   uri/fetchers/hadoop.hpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/tests/uri_fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/uri_fetcher_tests.cpp b/src/tests/uri_fetcher_tests.cpp
index dcfba78..072c09b 100644
--- a/src/tests/uri_fetcher_tests.cpp
+++ b/src/tests/uri_fetcher_tests.cpp
@@ -39,6 +39,7 @@
 #include "uri/fetcher.hpp"
 
 #include "uri/schemes/docker.hpp"
+#include "uri/schemes/file.hpp"
 #include "uri/schemes/hdfs.hpp"
 #include "uri/schemes/http.hpp"
 
@@ -292,6 +293,48 @@ TEST_F(DockerFetcherPluginTest, INTERNET_CURL_FetchImage)
 }
 
 
+class CopyFetcherPluginTest : public TemporaryDirectoryTest {};
+
+
+// Tests CopyFetcher plugin for fetching a valid file.
+TEST_F(CopyFetcherPluginTest, FetchExistingFile)
+{
+  const string file = path::join(os::getcwd(), "file");
+
+  ASSERT_SOME(os::write(file, "abc"));
+
+  // Create a URI for the test file.
+  const URI uri = uri::file(file);
+
+  // Use the file fetcher to fetch the URI.
+  Try<Owned<uri::Fetcher>> fetcher = uri::fetcher::create();
+  ASSERT_SOME(fetcher);
+
+  const string dir = path::join(os::getcwd(), "dir");
+
+  AWAIT_READY(fetcher.get()->fetch(uri, dir));
+
+  // Validate the fetched file's content.
+  EXPECT_SOME_EQ("abc", os::read(path::join(dir, "file")));
+}
+
+
+// Negative test case that tests CopyFetcher plugin for a non-exiting file.
+TEST_F(CopyFetcherPluginTest, FetchNonExistingFile)
+{
+  const URI uri = uri::file(path::join(os::getcwd(), "non-exist"));
+
+  // Use the file fetcher to fetch the URI.
+  Try<Owned<uri::Fetcher>> fetcher = uri::fetcher::create();
+  ASSERT_SOME(fetcher);
+
+  const string dir = path::join(os::getcwd(), "dir");
+
+  // Validate that the fetch failed.
+  AWAIT_FAILED(fetcher.get()->fetch(uri, dir));
+}
+
+
 // TODO(jieyu): Add Docker fetcher plugin tests to test with a local
 // registry server (w/ or w/o authentication).
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/uri/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/uri/fetcher.cpp b/src/uri/fetcher.cpp
index 2d8d5b1..8645b66 100644
--- a/src/uri/fetcher.cpp
+++ b/src/uri/fetcher.cpp
@@ -45,6 +45,7 @@ Try<Owned<Fetcher>> create(const Option<Flags>& _flags)
 
   hashmap<string, Creator> creators;
   creators.put("curl", lambda::bind(&CurlFetcherPlugin::create, flags));
+  creators.put("file", lambda::bind(&CopyFetcherPlugin::create, flags));
   creators.put("hadoop", lambda::bind(&HadoopFetcherPlugin::create, flags));
   creators.put("docker", lambda::bind(&DockerFetcherPlugin::create, flags));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/uri/fetcher.hpp
----------------------------------------------------------------------
diff --git a/src/uri/fetcher.hpp b/src/uri/fetcher.hpp
index 5bc9703..8af2c91 100644
--- a/src/uri/fetcher.hpp
+++ b/src/uri/fetcher.hpp
@@ -25,6 +25,7 @@
 
 #include <mesos/uri/fetcher.hpp>
 
+#include "uri/fetchers/copy.hpp"
 #include "uri/fetchers/curl.hpp"
 #include "uri/fetchers/docker.hpp"
 #include "uri/fetchers/hadoop.hpp"
@@ -37,6 +38,7 @@ namespace fetcher {
  * The combined flags for all built-in plugins.
  */
 class Flags :
+  public CopyFetcherPlugin::Flags,
   public CurlFetcherPlugin::Flags,
   public HadoopFetcherPlugin::Flags,
   public DockerFetcherPlugin::Flags {};

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/uri/fetchers/copy.cpp
----------------------------------------------------------------------
diff --git a/src/uri/fetchers/copy.cpp b/src/uri/fetchers/copy.cpp
new file mode 100644
index 0000000..2180adf
--- /dev/null
+++ b/src/uri/fetchers/copy.cpp
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <sys/wait.h>
+
+#include <string>
+#include <tuple>
+#include <vector>
+
+#include <process/check.hpp>
+#include <process/collect.hpp>
+#include <process/io.hpp>
+#include <process/subprocess.hpp>
+
+#include <stout/path.hpp>
+#include <stout/strings.hpp>
+
+#include <stout/os/mkdir.hpp>
+
+#include "uri/fetchers/copy.hpp"
+
+namespace io = process::io;
+
+using std::set;
+using std::string;
+using std::tuple;
+using std::vector;
+
+using process::await;
+using process::subprocess;
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+using process::Subprocess;
+
+namespace mesos {
+namespace uri {
+
+Try<Owned<Fetcher::Plugin>> CopyFetcherPlugin::create(const Flags& flags)
+{
+  return Owned<Fetcher::Plugin>(new CopyFetcherPlugin());
+}
+
+
+set<string> CopyFetcherPlugin::schemes()
+{
+  return {"file"};
+}
+
+
+Future<Nothing> CopyFetcherPlugin::fetch(
+    const URI& uri,
+    const string& directory)
+{
+  // TODO(jojy): Validate the given URI.
+
+  if (!uri.has_path()) {
+    return Failure("URI path is not specified");
+  }
+
+  // TODO(jojy): Verify that the path is a file.
+
+  Try<Nothing> mkdir = os::mkdir(directory);
+  if (mkdir.isError()) {
+    return Failure(
+        "Failed to create directory '" +
+        directory + "': " + mkdir.error());
+  }
+
+  VLOG(1) << "Copying '" << uri.path() << "' to '" << directory << "'";
+
+  const vector<string> argv = {"cp", "-a", uri.path(), directory};
+
+  Try<Subprocess> s = subprocess(
+      "cp",
+      argv,
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PIPE(),
+      Subprocess::PIPE());
+
+  if (s.isError()) {
+    return Failure("Failed to exec the copy subprocess: " + s.error());
+  }
+
+  return await(
+      s.get().status(),
+      io::read(s.get().out().get()),
+      io::read(s.get().err().get()))
+    .then([](const tuple<
+        Future<Option<int>>,
+        Future<string>,
+        Future<string>>& t) -> Future<Nothing> {
+      Future<Option<int>> status = std::get<0>(t);
+      if (!status.isReady()) {
+        return Failure(
+            "Failed to get the exit status of the copy subprocess: " +
+            (status.isFailed() ? status.failure() : "discarded"));
+      }
+
+      if (status->isNone()) {
+        return Failure("Failed to reap the copy subprocess");
+      }
+
+      if (status->get() != 0) {
+        Future<string> error = std::get<2>(t);
+        if (!error.isReady()) {
+          return Failure(
+              "Failed to perform 'copy'. Reading stderr failed: " +
+              (error.isFailed() ? error.failure() : "discarded"));
+        }
+
+        return Failure("Failed to perform 'copy': " + error.get());
+      }
+
+      return Nothing();
+    });
+}
+
+} // namespace uri {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fc113e68/src/uri/fetchers/copy.hpp
----------------------------------------------------------------------
diff --git a/src/uri/fetchers/copy.hpp b/src/uri/fetchers/copy.hpp
new file mode 100644
index 0000000..64e686c
--- /dev/null
+++ b/src/uri/fetchers/copy.hpp
@@ -0,0 +1,52 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __URI_FETCHERS_COPY_HPP__
+#define __URI_FETCHERS_COPY_HPP__
+
+#include <process/owned.hpp>
+
+#include <stout/flags.hpp>
+#include <stout/try.hpp>
+
+#include <mesos/uri/fetcher.hpp>
+
+namespace mesos {
+namespace uri {
+
+class CopyFetcherPlugin : public Fetcher::Plugin
+{
+public:
+  class Flags : public virtual flags::FlagsBase {};
+
+  static Try<process::Owned<Fetcher::Plugin>> create(const Flags& flags);
+
+  virtual ~CopyFetcherPlugin() {}
+
+  virtual std::set<std::string> schemes();
+
+  virtual process::Future<Nothing> fetch(
+      const URI& uri,
+      const std::string& directory);
+
+private:
+  CopyFetcherPlugin() {}
+};
+
+} // namespace uri {
+} // namespace mesos {
+
+#endif // __URI_FETCHERS_COPY_HPP__


[2/3] mesos git commit: Added support for file URI in Appc fetcher.

Posted by ji...@apache.org.
Added support for file URI in Appc fetcher.

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


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

Branch: refs/heads/master
Commit: 6903c6eeb8ffcbd1b40115d3f6f09198426f0adb
Parents: fc113e6
Author: Jojy Varghese <jo...@mesosphere.io>
Authored: Wed Mar 2 18:04:04 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Mar 2 18:04:04 2016 -0800

----------------------------------------------------------------------
 .../containerizer/mesos/provisioner/appc/fetcher.cpp     | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6903c6ee/src/slave/containerizer/mesos/provisioner/appc/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/provisioner/appc/fetcher.cpp b/src/slave/containerizer/mesos/provisioner/appc/fetcher.cpp
index e12a6f2..b29fc79 100644
--- a/src/slave/containerizer/mesos/provisioner/appc/fetcher.cpp
+++ b/src/slave/containerizer/mesos/provisioner/appc/fetcher.cpp
@@ -22,6 +22,7 @@
 #include "slave/containerizer/mesos/provisioner/appc/fetcher.hpp"
 #include "slave/containerizer/mesos/provisioner/appc/paths.hpp"
 
+#include "uri/schemes/file.hpp"
 #include "uri/schemes/http.hpp"
 
 namespace http = process::http;
@@ -84,6 +85,13 @@ static Try<URI> getUri(const string& prefix, const string& path)
   const string rawUrl = prefix + path;
 
   // TODO(jojy): Add parse URI function in URI namespace.
+
+  // TODO(jojy): Add `Path::seperator` which abstracts the file
+  // separator character.
+  if (strings::startsWith(rawUrl, "/")) {
+    return uri::file(rawUrl);
+  }
+
   Try<http::URL> _url = http::URL::parse(rawUrl);
   if (_url.isError()) {
     return Error(
@@ -132,7 +140,8 @@ Try<Owned<Fetcher>> Fetcher::create(
 
   // TODO(jojy): Add support for hdfs.
   if (!strings::startsWith(prefix, "http") &&
-      !strings::startsWith(prefix, "https")) {
+      !strings::startsWith(prefix, "https") &&
+      !strings::startsWith(prefix, "/")) {
     return Error("Invalid simple discovery uri prefix: " + prefix);
   }
 


[3/3] mesos git commit: Added test for Appc image fetcher.

Posted by ji...@apache.org.
Added test for Appc image fetcher.

Added simple appc Fetcher test with mock HTTP image server.

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


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

Branch: refs/heads/master
Commit: f0fb404616d7af58083396b880a2b14abb7e671b
Parents: 6903c6e
Author: Jojy Varghese <jo...@mesosphere.io>
Authored: Wed Mar 2 18:04:15 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Mar 2 18:04:15 2016 -0800

----------------------------------------------------------------------
 .../containerizer/provisioner_appc_tests.cpp    | 285 ++++++++++++++++---
 1 file changed, 252 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f0fb4046/src/tests/containerizer/provisioner_appc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/provisioner_appc_tests.cpp b/src/tests/containerizer/provisioner_appc_tests.cpp
index 9d9779a..86e1964 100644
--- a/src/tests/containerizer/provisioner_appc_tests.cpp
+++ b/src/tests/containerizer/provisioner_appc_tests.cpp
@@ -16,6 +16,8 @@
 
 #include <string>
 
+#include <gmock/gmock.h>
+
 #include <mesos/slave/isolator.hpp>
 
 #include <process/gtest.hpp>
@@ -31,11 +33,14 @@
 
 #include <mesos/appc/spec.hpp>
 
+#include "common/command_utils.hpp"
+
 #include "slave/paths.hpp"
 
 #include "slave/containerizer/mesos/provisioner/paths.hpp"
 #include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
+#include "slave/containerizer/mesos/provisioner/appc/fetcher.hpp"
 #include "slave/containerizer/mesos/provisioner/appc/paths.hpp"
 #include "slave/containerizer/mesos/provisioner/appc/store.hpp"
 
@@ -43,6 +48,9 @@ using std::list;
 using std::string;
 using std::vector;
 
+using testing::_;
+using testing::Return;
+
 using namespace process;
 
 using namespace mesos::internal::slave::appc;
@@ -133,8 +141,11 @@ protected:
   // directory. The directory structure reflects the Appc image spec.
   //
   // @param storeDir Path to the store directory where all images are stored.
+  // @param manifest Manifest JSON to be used for the test image.
   // @return  Path to the test image directory.
-  Try<string> createTestImage(const string& storeDir)
+  Try<string> createTestImage(
+      const string& storeDir,
+      const JSON::Value& manifest)
   {
     Try<Nothing> mkdir = os::mkdir(storeDir, true);
     if (mkdir.isError()) {
@@ -147,32 +158,6 @@ protected:
     //    |--<id>
     //       |--manifest
     //       |--rootfs/tmp/test
-    JSON::Value manifest = JSON::parse(
-        "{"
-        "  \"acKind\": \"ImageManifest\","
-        "  \"acVersion\": \"0.6.1\","
-        "  \"name\": \"foo.com/bar\","
-        "  \"labels\": ["
-        "    {"
-        "      \"name\": \"version\","
-        "      \"value\": \"1.0.0\""
-        "    },"
-        "    {"
-        "      \"name\": \"arch\","
-        "      \"value\": \"amd64\""
-        "    },"
-        "    {"
-        "      \"name\": \"os\","
-        "      \"value\": \"linux\""
-        "    }"
-        "  ],"
-        "  \"annotations\": ["
-        "    {"
-        "      \"name\": \"created\","
-        "      \"value\": \"1438983392\""
-        "    }"
-        "  ]"
-        "}").get();
 
     // The 'imageId' below has the correct format but it's not computed
     // by hashing the tarball of the image. It's OK here as we assume
@@ -247,6 +232,39 @@ protected:
 
     return appc;
   }
+
+  // Abstracts the manifest accessor for the test fixture. This provides the
+  // ability for customizing manifests for fixtures.
+  virtual JSON::Value getManifest() const
+  {
+    return JSON::parse(
+        R"~(
+        {
+          "acKind": "ImageManifest",
+          "acVersion": "0.6.1",
+          "name": "foo.com/bar",
+          "labels": [
+            {
+              "name": "version",
+              "value": "1.0.0"
+            },
+            {
+              "name": "arch",
+              "value": "amd64"
+            },
+            {
+              "name": "os",
+              "value": "linux"
+            }
+          ],
+          "annotations": [
+            {
+              "name": "created",
+              "value": "1438983392"
+            }
+          ]
+        })~").get();
+  }
 };
 
 
@@ -258,7 +276,9 @@ TEST_F(AppcStoreTest, Recover)
   Try<Owned<slave::Store>> store = Store::create(flags);
   ASSERT_SOME(store);
 
-  Try<string> createImage = createTestImage(flags.appc_store_dir);
+  Try<string> createImage = createTestImage(
+      flags.appc_store_dir,
+      getManifest());
 
   ASSERT_SOME(createImage);
 
@@ -296,12 +316,12 @@ TEST_F(ProvisionerAppcTest, ROOT_Provision)
   flags.image_provisioner_backend = "bind";
   flags.work_dir = "work_dir";
 
-  Fetcher fetcher;
-
   Try<Owned<Provisioner>> provisioner = Provisioner::create(flags);
   ASSERT_SOME(provisioner);
 
-  Try<string> createImage = createTestImage(flags.appc_store_dir);
+  Try<string> createImage = createTestImage(
+      flags.appc_store_dir,
+      getManifest());
 
   ASSERT_SOME(createImage);
 
@@ -363,11 +383,12 @@ TEST_F(ProvisionerAppcTest, Recover)
   flags.image_provisioner_backend = "copy";
   flags.work_dir = "work_dir";
 
-  Fetcher fetcher;
   Try<Owned<Provisioner>> provisioner1 = Provisioner::create(flags);
   ASSERT_SOME(provisioner1);
 
-  Try<string> createImage = createTestImage(flags.appc_store_dir);
+  Try<string> createImage = createTestImage(
+      flags.appc_store_dir,
+      getManifest());
 
   ASSERT_SOME(createImage);
 
@@ -428,6 +449,204 @@ TEST_F(ProvisionerAppcTest, Recover)
   EXPECT_FALSE(os::exists(containerDir));
 }
 
+
+// Mock HTTP image server.
+class TestAppcImageServer : public Process<TestAppcImageServer>
+{
+public:
+  TestAppcImageServer() : ProcessBase("TestAppcImageServer") {}
+
+  void addRoute(const string& imageName)
+  {
+    route("/" + imageName, None(), &TestAppcImageServer::serve);
+  }
+
+  MOCK_METHOD1(serve, Future<http::Response>(const http::Request&));
+};
+
+
+// Test fixture that uses the mock HTTP image server. This fixture provides the
+// abstraction for creating and composing complex test cases for Appc image
+// fetcher and store.
+class AppcImageFetcherTest : public AppcStoreTest
+{
+protected:
+  // Custom implementation that overrides the host and port of the image name.
+  JSON::Value getManifest() const
+  {
+    string imageName = strings::format(
+        "%s:%d/TestAppcImageServer/image",
+        stringify(server.self().address.ip),
+        server.self().address.port).get();
+
+    return JSON::parse(
+        R"~(
+        {
+          "acKind": "ImageManifest",
+          "acVersion": "0.6.1",
+          "name": " + imageName + ",
+          "labels": [
+            {
+              "name": "version",
+              "value": "1.0.0"
+            },
+            {
+              "name": "arch",
+              "value": "amd64"
+            },
+            {
+              "name": "os",
+              "value": "linux"
+            }
+          ],
+          "annotations": [
+            {
+              "name": "created",
+              "value": "1438983392"
+            }
+          ]
+        })~").get();
+  }
+
+  // TODO(jojy): Currently only supports serving one image. Consider adding
+  // support serving any image on the server. One way to do this is to add a map
+  // of image name -> server path for each image.
+  Future<http::Response> serveImage()
+  {
+    http::OK response;
+
+    response.type = response.PATH;
+    response.path = imageBundlePath;
+    response.headers["Content-Type"] = "application/octet-stream";
+    response.headers["Content-Disposition"] = strings::format(
+        "attachment; filename=%s",
+        imageBundlePath).get();
+
+    return response;
+  }
+
+  // TODO(jojy): Currently just uses 'imageId' to prepare a image on
+  // the server. Consider adding more parameters(e.g, 'labels').
+  void prepareServerImage(const string& fileName, const string& imageId)
+  {
+    const Path serverDir(path::join(os::getcwd(), "server"));
+
+    Try<string> createImage = createTestImage(serverDir, getManifest());
+    ASSERT_SOME(createImage);
+
+    // Set image file path for the test.
+    imageBundlePath = path::join(serverDir, fileName);
+
+    const Path imageDir(path::join(serverDir, "images", imageId));
+
+    Future<Nothing> future = command::tar(
+        Path("."),
+        Path(imageBundlePath),
+        imageDir,
+        command::Compression::GZIP);
+
+    AWAIT_READY(future);
+
+    // Now add route on the server for the image.
+    server.addRoute(fileName);
+  }
+
+  virtual void SetUp()
+  {
+    TemporaryDirectoryTest::SetUp();
+
+    // Now spawn the image server.
+    spawn(server);
+  }
+
+  virtual void TearDown()
+  {
+    terminate(server);
+    wait(server);
+
+    TemporaryDirectoryTest::TearDown();
+  }
+
+  string imageBundlePath;
+  TestAppcImageServer server;
+};
+
+
+// Tests simple fetch functionality of the appc::Fetcher component.
+// The test fetches a test Appc image from the http server and
+// verifies its content. The image is served in 'tar + gzip' format.
+TEST_F(AppcImageFetcherTest, SimpleFetch)
+{
+  // Setup the image on the image server.
+  prepareServerImage(
+      "image-latest-linux-amd64.aci",
+      "sha512-e77d96aa0240eedf134b8c90baeaf76dca8e78691836301d7498c8402044604"
+      "2e797b296d6ab296e0954c2626bfb264322ebeb8f447dac4fac6511ea06bc61f0");
+
+  // Setup server.
+  EXPECT_CALL(server, serve(_))
+    .WillOnce(Return(serveImage()));
+
+  // Appc Image to be fetched.
+  const string imageName =
+    stringify(server.self().address) + "/TestAppcImageServer/image";
+
+  Image::Appc imageInfo;
+  imageInfo.set_name(imageName);
+
+  Label archLabel;
+  archLabel.set_key("arch");
+  archLabel.set_value("amd64");
+
+  Label osLabel;
+  osLabel.set_key("os");
+  osLabel.set_value("linux");
+
+  Labels labels;
+  labels.add_labels()->CopyFrom(archLabel);
+  labels.add_labels()->CopyFrom(osLabel);
+
+  imageInfo.mutable_labels()->CopyFrom(labels);
+
+  // Create image fetcher.
+  Try<Owned<uri::Fetcher>> uriFetcher = uri::fetcher::create();
+  ASSERT_SOME(uriFetcher);
+
+  slave::Flags flags;
+
+  Try<Owned<slave::appc::Fetcher>> fetcher =
+    slave::appc::Fetcher::create(flags, uriFetcher.get().share());
+
+  ASSERT_SOME(fetcher);
+
+  // Prepare fetch directory.
+  const Path imageFetchDir(path::join(os::getcwd(), "fetched-images"));
+
+  Try<Nothing> mkdir = os::mkdir(imageFetchDir);
+  ASSERT_SOME(mkdir);
+
+  // Now fetch the image.
+  AWAIT_READY(fetcher.get()->fetch(imageInfo, imageFetchDir));
+
+  // Verify that there is an image directory.
+  Try<list<string>> imageDirs = os::ls(imageFetchDir);
+  ASSERT_SOME(imageDirs);
+
+  // Verify that there is only ONE image directory.
+  ASSERT_EQ(1u, imageDirs.get().size());
+
+  // Verify that there is a roofs.
+  const Path imageRootfs(path::join(
+      imageFetchDir,
+      imageDirs.get().front(),
+      "rootfs"));
+
+  ASSERT_TRUE(os::exists(imageRootfs));
+
+  // Verify that the image fetched is the same as on the server.
+  ASSERT_SOME_EQ("test", os::read(path::join(imageRootfs, "tmp", "test")));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {