You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/08/09 00:50:18 UTC

svn commit: r1371003 - in /incubator/mesos/trunk: src/Makefile.am src/files/ src/files/files.cpp src/files/files.hpp src/master/http.cpp third_party/libprocess/include/stout/os.hpp third_party/libprocess/include/stout/strings.hpp

Author: benh
Date: Wed Aug  8 22:50:17 2012
New Revision: 1371003

URL: http://svn.apache.org/viewvc?rev=1371003&view=rev
Log:
Added a 'files' abstraction for browsing and reading files from HTTP
endpoints (https://reviews.apache.org/r/6477).

Added:
    incubator/mesos/trunk/src/files/
    incubator/mesos/trunk/src/files/files.cpp
    incubator/mesos/trunk/src/files/files.hpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/master/http.cpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1371003&r1=1371002&r2=1371003&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Wed Aug  8 22:50:17 2012
@@ -159,9 +159,10 @@ libmesos_no_third_party_la_SOURCES = sch
 	launcher/launcher.cpp exec/exec.cpp common/lock.cpp		\
 	detector/detector.cpp configurator/configurator.cpp		\
 	common/date_utils.cpp common/resources.cpp			\
-	common/attributes.cpp common/values.cpp logging/logging.cpp	\
-	zookeeper/zookeeper.cpp zookeeper/authentication.cpp		\
-	zookeeper/group.cpp messages/messages.proto
+	common/attributes.cpp common/values.cpp files/files.cpp		\
+	logging/logging.cpp zookeeper/zookeeper.cpp			\
+	zookeeper/authentication.cpp zookeeper/group.cpp		\
+	messages/messages.proto
 
 pkginclude_HEADERS = $(top_srcdir)/include/mesos/executor.hpp	\
 		     $(top_srcdir)/include/mesos/scheduler.hpp	\
@@ -191,7 +192,7 @@ libmesos_no_third_party_la_SOURCES += co
 	common/type_utils.hpp common/thread.hpp common/units.hpp	\
 	common/values.hpp configurator/configuration.hpp		\
 	configurator/configurator.hpp configurator/option.hpp		\
-	detector/detector.hpp examples/utils.hpp flags/flag.hpp		\
+	detector/detector.hpp examples/utils.hpp files/files.hpp	\
 	flags/flags.hpp flags/loader.hpp launcher/launcher.hpp		\
 	linux/cgroups.hpp linux/fs.hpp linux/proc.hpp local/flags.hpp	\
 	local/local.hpp logging/flags.hpp logging/logging.hpp		\

Added: incubator/mesos/trunk/src/files/files.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/files/files.cpp?rev=1371003&view=auto
==============================================================================
--- incubator/mesos/trunk/src/files/files.cpp (added)
+++ incubator/mesos/trunk/src/files/files.cpp Wed Aug  8 22:50:17 2012
@@ -0,0 +1,286 @@
+#include <map>
+#include <string>
+#include <vector>
+
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/process.hpp>
+
+#include <stout/hashmap.hpp>
+#include <stout/json.hpp>
+#include <stout/numify.hpp>
+#include <stout/option.hpp>
+#include <stout/os.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+#include <stout/try.hpp>
+
+#include "files/files.hpp"
+
+#include "logging/logging.hpp"
+
+using namespace process;
+
+using process::http::BadRequest;
+using process::http::InternalServerError;
+using process::http::NotFound;
+using process::http::OK;
+using process::http::Response;
+using process::http::Request;
+using process::http::ServiceUnavailable;
+
+using std::map;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+
+class FilesProcess : public Process<FilesProcess>
+{
+public:
+  FilesProcess() {}
+  virtual ~FilesProcess() {}
+
+  // Files implementation.
+  Future<bool> attach(const string& path, const string& name);
+  void detach(const string& name);
+
+protected:
+  virtual void initialize();
+
+private:
+  // JSON endpoints.
+  Future<Response> browse(const Request& request);
+  Future<Response> read(const Request& request);
+
+  hashmap<string, string> paths;
+};
+
+
+void FilesProcess::initialize()
+{
+  route("/browse.json", &FilesProcess::browse);
+  route("/read.json", &FilesProcess::read);
+}
+
+
+Future<bool> FilesProcess::attach(const string& path, const string& name)
+{
+  Try<string> result = os::realpath(path);
+
+  if (result.isError()) {
+    return Future<bool>::failed(result.error());
+  }
+
+  // Make sure we have permissions to "touch" the file (TODO(benh): We
+  // really only need permissions to read the file).
+  Try<bool> touched = os::touch(result.get());
+
+  if (touched.isError()) {
+    return Future<bool>::failed(touched.error());
+  }
+
+  if (touched.get()) {
+    paths[name] = result.get();
+  }
+
+  return touched.get();
+}
+
+
+void FilesProcess::detach(const string& name)
+{
+  paths.erase(name);
+}
+
+
+Future<Response> FilesProcess::browse(const Request& request)
+{
+  return ServiceUnavailable();
+}
+
+
+Future<Response> FilesProcess::read(const Request& request)
+{
+  map<string, vector<string> > pairs =
+    strings::pairs(request.query, ";&", "=");
+
+  Option<string> name;
+
+  if (pairs.count("name") > 0 && pairs["name"].size() > 0) {
+    name = pairs["name"].back();
+  }
+
+  if (!name.isSome()) {
+    return BadRequest();
+  }
+
+  off_t offset = -1;
+
+  if (pairs.count("offset") > 0 && pairs["offset"].size() > 0) {
+    Try<off_t> result = numify<off_t>(pairs["offset"].back());
+    if (result.isError()) {
+      LOG(WARNING) << "Failed to \"numify\" the 'offset' ("
+                   << pairs["offset"].back() << "): "
+                   << result.error();
+      return InternalServerError();
+    }
+    offset = result.get();
+  }
+
+  ssize_t length = -1;
+
+  if (pairs.count("length") > 0) {
+    CHECK(pairs["length"].size() > 0);
+    Try<ssize_t> result = numify<ssize_t>(pairs["length"].back());
+    if (result.isError()) {
+      LOG(WARNING) << "Failed to \"numify\" the 'length' ("
+                   << pairs["length"].back() << "): "
+                   << result.error();
+      return InternalServerError();
+    }
+    length = result.get();
+  }
+
+  // Now try and see if this name has been attached. We check for the
+  // longest possible prefix match and if found append any suffix to
+  // the attached path (provided the path is to a directory).
+  string suffix;
+
+  foreach (const string& s, strings::split(name.get(), "/")) {
+    string prefix = name.get().substr(0, name.get().length() - suffix.length());
+
+    if (!paths.contains(prefix)) {
+      suffix = s + "/" + suffix;
+    } else {
+      // Determine the final path: if it's a directory, append the
+      // suffix, if it's not a directory and there is a suffix, return
+      // '404 Not Found'.
+      string path;
+      if (os::exists(paths[prefix], true)) {
+        path = paths[prefix] + "/" + suffix;
+
+        // Canonicalize the absolute path and make sure the result
+        // doesn't break out of the chroot (i.e., resolving any '..'
+        // in the suffix should yield a resulting path that still
+        // contains the attached path as it's prefix).
+        Try<string> result = os::realpath(path);
+        if (result.isError()) {
+          return NotFound();
+        } else if (result.get().find(paths[prefix]) != 0) {
+          return NotFound();
+        }
+
+        path = result.get();
+      } else if (suffix != "") {
+        return NotFound();
+      }
+
+      // TODO(benh): Cache file descriptors so we aren't constantly
+      // opening them and paging the data in from disk.
+      Try<int> fd = os::open(path, O_RDONLY);
+
+      if (fd.isError()) {
+        LOG(WARNING) << "Failed to open file at "
+                     << path << ": " << fd.error();
+        return InternalServerError();
+      }
+
+      off_t size = lseek(fd.get(), 0, SEEK_END);
+
+      if (size == -1) {
+        PLOG(WARNING) << "Failed to seek in the file at " << path;
+        close(fd.get());
+        return InternalServerError();
+      }
+
+      if (offset == -1) {
+        offset = size;
+      }
+
+      if (length == -1) {
+        length = size - offset;
+      }
+
+      JSON::Object object;
+
+      if (offset < size) {
+        // Seek to the offset we want to read from.
+        if (lseek(fd.get(), offset, SEEK_SET) == -1) {
+          PLOG(WARNING) << "Failed to seek in the file at " << path;
+          close(fd.get());
+          return InternalServerError();
+        }
+
+        // Read length bytes (or to EOF).
+        char* temp = new char[length];
+
+        length = ::read(fd.get(), temp, length);
+
+        if (length == 0) {
+          object.values["offset"] = offset;
+          object.values["length"] = 0;
+          delete[] temp;
+        } else if (length == -1) {
+          PLOG(WARNING) << "Failed to read from the file at " << path;
+          delete[] temp;
+          close(fd.get());
+          return InternalServerError();
+        } else {
+          object.values["offset"] = offset;
+          object.values["length"] = length;
+          object.values["data"] = string(temp, length);
+          delete[] temp;
+        }
+      } else {
+        object.values["offset"] = size;
+        object.values["length"] = 0;
+      }
+
+      close(fd.get());
+
+      std::ostringstream out;
+
+      JSON::render(out, object);
+
+      OK response;
+      response.headers["Content-Type"] = "application/json";
+      response.headers["Content-Length"] = stringify(out.str().size());
+      response.body = out.str().data();
+      return response;
+    }
+  }
+
+  return NotFound();
+}
+
+
+Files::Files()
+{
+  process = new FilesProcess();
+  spawn(process);
+}
+
+
+Files::~Files()
+{
+  terminate(process);
+  wait(process);
+}
+
+
+Future<bool> Files::attach(const string& path, const string& name)
+{
+  return dispatch(process, &FilesProcess::attach, path, name);
+}
+
+
+void Files::detach(const string& name)
+{
+  dispatch(process, &FilesProcess::detach, name);
+}
+
+} // namespace internal {
+} // namespace mesos {

Added: incubator/mesos/trunk/src/files/files.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/files/files.hpp?rev=1371003&view=auto
==============================================================================
--- incubator/mesos/trunk/src/files/files.hpp (added)
+++ incubator/mesos/trunk/src/files/files.hpp Wed Aug  8 22:50:17 2012
@@ -0,0 +1,61 @@
+/**
+ * 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 __FILES_HPP__
+#define __FILES_HPP__
+
+#include <string>
+
+#include <process/future.hpp>
+
+namespace mesos {
+namespace internal {
+
+// Forward declarations.
+class FilesProcess;
+
+
+// Provides an abstraction for browsing and reading files via HTTP
+// endpoints. A path (file or directory) may be "attached" to a name
+// (similar to "mounting" a device) for subsequent browsing and
+// reading of any files and directories it contains. The "mounting" of
+// paths to names enables us to do a form of chrooting for better
+// security and isolation of files.
+class Files
+{
+public:
+  Files();
+  ~Files();
+
+  // Returns the result of trying to attach the specified path
+  // (directory or file) at the specified name.
+  process::Future<bool> attach(
+      const std::string& path,
+      const std::string& name);
+
+  // Removes the specified name.
+  void detach(const std::string& name);
+
+private:
+  FilesProcess* process;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __FILES_HPP__

Modified: incubator/mesos/trunk/src/master/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/http.cpp?rev=1371003&r1=1371002&r2=1371003&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/http.cpp (original)
+++ incubator/mesos/trunk/src/master/http.cpp Wed Aug  8 22:50:17 2012
@@ -352,7 +352,7 @@ Future<Response> log(
 
   off_t offset = -1;
   ssize_t length = -1;
-  string level = pairs.count("level") > 0 ? pairs["level"][0] : "INFO";
+  string level = "INFO";
 
   if (pairs.count("offset") > 0 && pairs["offset"].size() > 0) {
     Try<off_t> result = numify<off_t>(pairs["offset"].back());
@@ -376,6 +376,10 @@ Future<Response> log(
     length = result.get();
   }
 
+  if (pairs.count("level") > 0 && pairs["level"].size() > 0) {
+    level = pairs["level"].back();
+  }
+
   if (master.flags.log_dir.isNone()) {
     return NotFound();
   }

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp?rev=1371003&r1=1371002&r2=1371003&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp Wed Aug  8 22:50:17 2012
@@ -155,6 +155,10 @@ inline Try<bool> touch(const std::string
     return Try<bool>::error("Failed to open file " + path);
   }
 
+  // TODO(benh): Is opening/closing sufficient to have the same
+  // semantics as the touch utility (i.e., doesn't the utility change
+  // the modified date)?
+
   Try<bool> result = close(fd.get());
 
   if (result.isError()) {

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp?rev=1371003&r1=1371002&r2=1371003&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp Wed Aug  8 22:50:17 2012
@@ -166,7 +166,6 @@ inline std::string upper(const std::stri
   return result;
 }
 
-
 } // namespaces strings {
 
 #endif // __STOUT_STRINGS_HPP__