Pārlūkot izejas kodu

HDFS-10874: libhdfs++: Public API headers should not depend on internal implementation. Contributed by James Clampffer

James Clampffer 7 gadi atpakaļ
vecāks
revīzija
0c7aa704ea
36 mainītis faili ar 523 papildinājumiem un 473 dzēšanām
  1. 11 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h
  3. 137 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/uri.h
  4. 11 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.h
  6. 4 3
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.cc
  7. 4 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
  8. 7 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
  9. 109 55
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/uri.cc
  10. 0 130
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/uri.h
  11. 5 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
  12. 144 95
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/uri_test.cc
  13. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_allowSnapshot.cc
  14. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_cat.cc
  15. 4 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chgrp.cc
  16. 4 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chmod.cc
  17. 4 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chown.cc
  18. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_copyToLocal.cc
  19. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_count.cc
  20. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_createSnapshot.cc
  21. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_deleteSnapshot.cc
  22. 2 6
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_df.cc
  23. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_disallowSnapshot.cc
  24. 4 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_du.cc
  25. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc
  26. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc
  27. 4 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc
  28. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_mkdir.cc
  29. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_moveToLocal.cc
  30. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_renameSnapshot.cc
  31. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_rm.cc
  32. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc
  33. 3 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc
  34. 5 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc
  35. 14 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/tools_common.cc
  36. 3 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/tools_common.h

+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h

@@ -25,6 +25,7 @@
 #include "hdfspp/statinfo.h"
 #include "hdfspp/fsinfo.h"
 #include "hdfspp/content_summary.h"
+#include "hdfspp/uri.h"
 
 #include <functional>
 #include <memory>
@@ -199,9 +200,17 @@ class FileSystem {
    **/
   static FileSystem *New();
 
+  /**
+   *  Callback type for async FileSystem::Connect calls.
+   *    Provides the result status and instance pointer to the handler.
+   **/
+  typedef std::function<void(const Status& result_status, FileSystem *created_fs)> AsyncConnectCallback;
 
+  /**
+   *  Connect directly to the specified namenode using the host and port (service).
+   **/
   virtual void Connect(const std::string &server, const std::string &service,
-      const std::function<void(const Status &, FileSystem *)> &handler) = 0;
+      const AsyncConnectCallback &handler) = 0;
 
   /* Synchronous call of Connect */
   virtual Status Connect(const std::string &server, const std::string &service) = 0;
@@ -214,7 +223,7 @@ class FileSystem {
    * If no defaultFs is defined, returns an error.
    */
   virtual void ConnectToDefaultFs(
-      const std::function<void(const Status &, FileSystem *)> &handler) = 0;
+      const AsyncConnectCallback& handler) = 0;
   virtual Status ConnectToDefaultFs() = 0;
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h

@@ -18,7 +18,7 @@
 #ifndef LIBHDFSPP_OPTIONS_H_
 #define LIBHDFSPP_OPTIONS_H_
 
-#include "common/uri.h"
+#include "hdfspp/uri.h"
 
 #include <string>
 #include <vector>

+ 137 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/uri.h

@@ -0,0 +1,137 @@
+/**
+ * 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 COMMON_HDFS_URI_H_
+#define COMMON_HDFS_URI_H_
+
+#include <iostream>
+#include <string>
+#include <vector>
+#include <stdexcept>
+
+namespace hdfs
+{
+
+class uri_parse_error : public std::invalid_argument {
+ public:
+  uri_parse_error(const char *what_str) : std::invalid_argument(what_str) {}
+  uri_parse_error(const std::string& what_str) : std::invalid_argument(what_str) {}
+};
+
+class URI {
+public:
+  // Parse a string into a URI.  Throw a hdfs::uri_parse_error if URI is malformed.
+  static URI parse_from_string(const std::string &str);
+
+  // URI encode/decode strings
+  static std::string encode  (const std::string &input);
+  static std::string decode  (const std::string &input);
+
+  URI();
+
+  std::string get_scheme(bool encoded_output=false) const;
+
+  void set_scheme(const std::string &s, bool encoded_input=false);
+
+  // empty if none.
+  std::string get_host(bool encoded_output=false) const;
+
+  void set_host(const std::string& h, bool encoded_input=false);
+
+  // true if port has been set
+  bool has_port() const;
+
+  // undefined if port hasn't been set
+  uint16_t get_port() const;
+
+  // use default if port hasn't been set
+  uint16_t get_port_or_default(uint16_t default_val) const;
+
+  void set_port(uint16_t p);
+
+  void clear_port();
+
+  std::string get_path(bool encoded_output=false) const;
+
+  void set_path(const std::string &p, bool encoded_input=false);
+
+  void add_path(const std::string &p, bool encoded_input=false);
+
+  std::vector<std::string> get_path_elements(bool encoded_output=false) const;
+
+  struct Query {
+    Query(const std::string& key, const std::string& val);
+    std::string key;
+    std::string value;
+  };
+
+  std::string get_query(bool encoded_output=false) const;
+
+  std::vector<Query> get_query_elements(bool encoded_output=false) const;
+
+  // Not that set_query must always pass in encoded strings
+  void set_query(const std::string &q);
+
+  // Adds a parameter onto the query; does not check if it already exists
+  //   e.g. parseFromString("foo?bar=baz").addQuery("bing","bang")
+  //   would leave "bar=baz&bing=bang" as the query
+  void add_query(const std::string &name, const std::string & value, bool encoded_input=false);
+
+  // Removes the query part if exists
+  //   e.g. parseFromString("foo?bar=baz&bing=bang&bar=bong").removeQueries("bar")
+  //   would leave bing=bang as the query
+  void remove_query(const std::string &q_name, bool encoded_input=false);
+
+  std::string get_fragment(bool encoded_output=false) const;
+
+  void set_fragment(const std::string &f, bool encoded_input=false);
+
+  std::string str(bool encoded_output=true) const;
+
+  // Get a string with each URI field printed on a seperate line
+  std::string GetDebugString() const;
+private:
+  // These are stored in encoded form
+  std::string scheme;
+  std::string user;
+  std::string pass;
+  std::string host;
+  std::vector<std::string> path;
+  std::vector<Query> queries;
+  std::string fragment;
+  // implicitly narrowed to uint16_t if positive
+  // -1 to indicate uninitialized
+  int32_t _port;
+
+  // URI encoding helpers
+  static std::string from_encoded(bool encoded_output, const std::string & input);
+  static std::string to_encoded(bool encoded_input, const std::string & input);
+
+  bool has_authority() const;
+  std::string build_authority(bool encoded_output) const;
+
+  std::string build_path(bool encoded_output) const;
+  void parse_path(bool input_encoded, const std::string &input_path);
+};
+
+inline std::ostream& operator<<(std::ostream &out, const URI &uri) {
+  return out << uri.str();
+}
+
+}
+#endif

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc

@@ -32,7 +32,7 @@
  */
 
 #include "configuration.h"
-#include "uri.h"
+#include "hdfspp/uri.h"
 
 #include <strings.h>
 #include <sstream>
@@ -140,12 +140,15 @@ bool Configuration::GetBoolWithDefault(const std::string& key,
 }
 
 optional<URI> Configuration::GetUri(const std::string& key) const {
-  auto raw = Get(key);
+  optional<std::string> raw = Get(key);
   if (raw) {
-    return URI::parse_from_string(*raw);
-  } else {
-    return optional<URI>();
+    try {
+      return std::experimental::make_optional(URI::parse_from_string(*raw));
+    } catch (const uri_parse_error& e) {
+      // Return empty below
+    }
   }
+  return optional<URI>();
 }
 
 URI Configuration::GetUriWithDefault(const std::string& key,
@@ -154,10 +157,9 @@ URI Configuration::GetUriWithDefault(const std::string& key,
   if (result) {
     return *result;
   } else {
-    result = URI::parse_from_string(default_value);
-    if (result) {
-      return *result;
-    } else {
+    try {
+      return URI::parse_from_string(default_value);
+    } catch (const uri_parse_error& e) {
       return URI();
     }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.h

@@ -19,7 +19,7 @@
 #ifndef COMMON_CONFIGURATION_H_
 #define COMMON_CONFIGURATION_H_
 
-#include "common/uri.h"
+#include "hdfspp/uri.h"
 
 #include <string>
 #include <map>

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.cc

@@ -131,13 +131,14 @@ std::vector<NamenodeInfo> HdfsConfiguration::LookupNameService(const std::string
     for(auto node_id=namenode_ids.begin(); node_id != namenode_ids.end(); node_id++) {
       // find URI
       std::string dom_node_name = std::string("dfs.namenode.rpc-address.") + nameservice + "." + *node_id;
-      optional<URI> node_uri = URI::parse_from_string(PrependHdfsScheme(Get(dom_node_name)));
 
-      if(!node_uri) {
+      URI uri;
+      try {
+        uri = URI::parse_from_string(PrependHdfsScheme(Get(dom_node_name)));
+      } catch (const uri_parse_error) {
         throw ha_parse_error("unable to find " + dom_node_name);
       }
 
-      URI uri = node_uri.value();
       if(uri.str() == "") {
         LOG_WARN(kRPC, << "Attempted to read info for nameservice " << nameservice << " node " << dom_node_name << " but didn't find anything.")
       } else {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc

@@ -41,11 +41,11 @@ std::string ResolvedNamenodeInfo::str() const {
   std::stringstream ss;
   ss << "ResolvedNamenodeInfo {nameservice: " << nameservice << ", name: " << name << ", uri: " << uri.str();
   ss << ", host: " << uri.get_host();
-  auto port = uri.get_port();
-  if(port)
-    ss << ", port: " << port.value();
+
+  if(uri.has_port())
+    ss << ", port: " << uri.get_port();
   else
-    ss << ", port: unable to parse";
+    ss << ", invalid port (uninitialized)";
 
   ss << ", scheme: " << uri.get_scheme();
 

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc

@@ -48,11 +48,14 @@ Options::Options() : rpc_timeout(kDefaultRpcTimeout),
 std::string NamenodeInfo::get_host() const {
   return uri.get_host();
 }
+
 std::string NamenodeInfo::get_port() const {
-  optional<uint16_t> p = uri.get_port();
-  if(!p)
-    return std::to_string(-1);
-  return std::to_string(p.value());
+  if(uri.has_port()) {
+    return std::to_string(uri.get_port());
+  }
+  return "-1";
 }
 
+
+
 }

+ 109 - 55
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/uri.cc

@@ -17,17 +17,16 @@
  */
 
 
-#include <common/uri.h>
+#include <hdfspp/uri.h>
 
 #include <uriparser2/uriparser/Uri.h>
 
 #include <string.h>
 #include <sstream>
 #include <cstdlib>
+#include <cassert>
 #include <limits>
 
-using std::experimental::nullopt;
-
 namespace hdfs
 {
 
@@ -140,25 +139,23 @@ std::string copy_range(const UriTextRangeA *r) {
   return "";
 }
 
-bool parse_int(const UriTextRangeA *r, optional<uint16_t> * result) {
-  assert(result); // output
-  std::string int_string = copy_range(r);
-  if (!int_string.empty()) {
+bool parse_int(const UriTextRangeA *r, int32_t& result)
+{
+  std::string int_str = copy_range(r);
+  if(!int_str.empty()) {
     errno = 0;
-    unsigned long val = ::strtoul(int_string.c_str(), nullptr, 10);
-    if (errno == 0 && val < std::numeric_limits<uint16_t>::max() ) {
-      *result = std::experimental::make_optional<uint16_t>(val);
+    unsigned long val = ::strtoul(int_str.c_str(), nullptr, 10);
+    if(errno == 0 && val < std::numeric_limits<uint16_t>::max()) {
+      result = val;
       return true;
     } else {
       return false;
     }
   }
-
-  // No value
-  *result = nullopt;
   return true;
 }
 
+
 std::vector<std::string> copy_path(const UriPathSegmentA *ps) {
     std::vector<std::string> result;
   if (nullptr == ps)
@@ -190,8 +187,8 @@ void parse_user_info(const UriTextRangeA *r, std::string * user, std::string * p
 }
 
 
-std::vector<std::pair<std::string, std::string > > parse_query(const char *first, const char * afterLast) {
-    std::vector<std::pair<std::string, std::string > >  result;
+std::vector<URI::Query> parse_queries(const char *first, const char * afterLast) {
+    std::vector<URI::Query>  result;
     UriQueryListA * query;
     int count;
     int dissect_result = uriDissectQueryMallocExA(&query, &count, first, afterLast, false, URI_BR_DONT_TOUCH);
@@ -199,7 +196,7 @@ std::vector<std::pair<std::string, std::string > > parse_query(const char *first
       for (auto ps = query; ps != nullptr; ps = ps->next) {
         std::string key = ps->key ? URI::encode(ps->key) : "";
         std::string value = ps->value ? URI::encode(ps->value) : "";
-          result.push_back(std::make_pair(key, value));
+          result.emplace_back(key, value);
       }
       uriFreeQueryListA(query);
     }
@@ -207,8 +204,8 @@ std::vector<std::pair<std::string, std::string > > parse_query(const char *first
   return result;
 }
 
-
-optional<URI> URI::parse_from_string(const std::string &str)
+// Parse a string into a URI.  Throw a hdfs::uri_parse_error if URI is malformed.
+URI URI::parse_from_string(const std::string &str)
 {
   URI ret;
   bool ok = true;
@@ -224,9 +221,9 @@ optional<URI> URI::parse_from_string(const std::string &str)
   if (ok) {
     ret.scheme = copy_range(&uu.scheme);
     ret.host = copy_range(&uu.hostText);
-    ok &= parse_int(&uu.portText, &ret.port);
+    ok &= parse_int(&uu.portText, ret._port);
     ret.path = copy_path(uu.pathHead);
-    ret.query = parse_query(uu.query.first, uu.query.afterLast);
+    ret.queries = parse_queries(uu.query.first, uu.query.afterLast);
     ret.fragment = copy_range(&uu.fragment);
     parse_user_info(&uu.userInfo, &ret.user, &ret.pass);
     uriFreeUriMembersA(&uu);
@@ -234,9 +231,9 @@ optional<URI> URI::parse_from_string(const std::string &str)
   uriFreeUriMembersA(&uu);
 
   if (ok) {
-    return std::experimental::make_optional(ret);
+    return ret;
   } else {
-    return nullopt;
+    throw uri_parse_error(str);
   }
 }
 
@@ -246,6 +243,9 @@ optional<URI> URI::parse_from_string(const std::string &str)
 //
 ///////////////////////////////////////////////////////////////////////////////
 
+URI::URI() : _port(-1) {}
+URI::Query::Query(const std::string& k, const std::string& v) : key(k), value(v) {}
+
 std::string URI::str(bool encoded_output) const
 {
   std::stringstream ss;
@@ -257,7 +257,7 @@ std::string URI::str(bool encoded_output) const
   }
   if (has_authority()) ss << build_authority(encoded_output);
   if (!path.empty()) ss << get_path(encoded_output);
-  if (!query.empty()) ss << "?" << get_query(encoded_output);
+  if (!queries.empty()) ss << "?" << get_query(encoded_output);
   if (!fragment.empty()) ss << "#" << from_encoded(encoded_output, fragment);
 
   return ss.str();
@@ -265,25 +265,62 @@ std::string URI::str(bool encoded_output) const
 
 bool URI::has_authority() const
 {
-  return (!host.empty()) || (port);
+  return (!host.empty()) || (has_port());
 }
 
 std::string URI::build_authority(bool encoded_output) const
 {
   std::stringstream ss;
   ss << URI::from_encoded(encoded_output, host);
-  if (port)
+  if (has_port())
   {
-    ss << ":" << *port;
+    ss << ":" << _port;
   }
   return ss.str();
 }
 
+std::string URI::get_scheme(bool encoded_output) const {
+  return from_encoded(encoded_output,scheme);
+}
+
+void URI::set_scheme(const std::string &s, bool encoded_input) {
+  scheme = to_encoded(encoded_input,s);
+}
+
+std::string URI::get_host(bool encoded_output) const {
+  return from_encoded(encoded_output,host);
+}
+
+void URI::set_host(const std::string& h, bool encoded_input) {
+  host = to_encoded(encoded_input,h);
+}
+
+bool URI::has_port() const {
+  return _port != -1;
+}
+
+uint16_t URI::get_port() const {
+  return (uint16_t)_port;
+}
+
+uint16_t URI::get_port_or_default(uint16_t val) const {
+  return has_port() ? (uint16_t)_port : val;
+}
+
+void URI::set_port(uint16_t p)
+{
+  _port = (int32_t)p & 0xFFFF;
+}
+
+void URI::clear_port()
+{
+  _port = -1;
+}
 
 std::string URI::get_path(bool encoded_output) const
 {
   std::ostringstream out;
-  for (auto s: path) {
+  for (const std::string& s: path) {
     out << "/" << from_encoded(encoded_output, s);
   }
   return out.str();
@@ -292,7 +329,7 @@ std::string URI::get_path(bool encoded_output) const
 std::vector<std::string> URI::get_path_elements(bool encoded_output) const
 {
   std::vector<std::string> result;
-  for (auto path_elem: path) {
+  for (const std::string& path_elem: path) {
     result.push_back(from_encoded(encoded_output, path_elem));
   }
 
@@ -302,72 +339,89 @@ std::vector<std::string> URI::get_path_elements(bool encoded_output) const
 void URI::parse_path(bool input_encoded, const std::string &input_path)
 {
   std::vector<std::string> split_path = split(input_path, '/');
-  for (auto s: split_path) {
+  for (const std::string& s: split_path) {
     path.push_back(to_encoded(input_encoded, s));
   }
 }
 
-
 // Mostly copied and modified from uriparser2.c
 
+void URI::set_path(const std::string &p, bool encoded_input) {
+  parse_path(encoded_input, p);
+}
+
 void URI::add_path(const std::string &p, bool encoded_input)
 {
   path.push_back(to_encoded(encoded_input, p));
 }
 
-
 std::string URI::get_query(bool encoded_output) const {
   bool first = true;
   std::stringstream ss;
-  for (auto q: query) {
+  for (const Query& q: queries) {
     if (!first) {
       ss << "&";
     }
-    ss << from_encoded(encoded_output, q.first) << "=" << from_encoded(encoded_output, q.second);
+    ss << from_encoded(encoded_output, q.key) << "=" << from_encoded(encoded_output, q.value);
     first = false;
   }
 
   return ss.str();
 }
 
-std::vector< std::pair<std::string, std::string> > URI::get_query_elements(bool encoded_output) const
+std::vector<URI::Query> URI::get_query_elements(bool encoded_output) const
 {
-  std::vector< std::pair<std::string, std::string> > result;
-  for (auto q: query) {
-    auto key = from_encoded(encoded_output, q.first);
-    auto value = from_encoded(encoded_output, q.second);
-    result.push_back(std::make_pair(key, value));
+  std::vector<Query> result;
+  for (const Query& q: queries) {
+    std::string key = from_encoded(encoded_output, q.key);
+    std::string value = from_encoded(encoded_output, q.value);
+    result.emplace_back(key, value);
   }
 
   return result;
 }
 
-
 void URI::set_query(const std::string &q) {
-  query = parse_query(q.c_str(), q.c_str() + q.size() + 1);
+  queries = parse_queries(q.c_str(), q.c_str() + q.size() + 1);
 }
 
 
 void URI::add_query(const std::string &name, const std::string & value, bool encoded_input)
 {
-  query.push_back(std::make_pair(to_encoded(encoded_input, name), to_encoded(encoded_input, value)));
+  queries.emplace_back(to_encoded(encoded_input, name), to_encoded(encoded_input, value));
 }
 
-void URI::remove_queries(const std::string &q_name, bool encoded_input)
+void URI::remove_query(const std::string &q_name, bool encoded_input)
 {
-  if (query.empty())
+  if (queries.empty())
     return;
 
   // This is the one place we need to do decoded comparisons
   std::string decoded_key = encoded_input ? decode(q_name) : q_name;
 
-  for (int i = query.size() - 1; i >= 0; i--) {
-    if (decode(query[i].first) == decoded_key) {
-      query.erase(query.begin() + i);
+  for (int i = queries.size() - 1; i >= 0; i--) {
+    if (decode(queries[i].key) == decoded_key) {
+      queries.erase(queries.begin() + i);
     }
   }
 }
 
+std::string URI::get_fragment(bool encoded_output) const {
+  return from_encoded(encoded_output, fragment);
+}
+
+void URI::set_fragment(const std::string &f, bool encoded_input) {
+  fragment = to_encoded(encoded_input,f);
+}
+
+std::string URI::from_encoded(bool encoded_output, const std::string & input) {
+  return encoded_output ? input : decode(input);
+}
+
+std::string URI::to_encoded(bool encoded_input, const std::string & input) {
+  return encoded_input ? input : encode(input);
+}
+
 std::string URI::GetDebugString() const {
   std::stringstream ss;
   ss << std::endl;
@@ -375,25 +429,25 @@ std::string URI::GetDebugString() const {
   ss << "\t" << "uri.get_scheme() = \"" << get_scheme() << "\"" << std::endl;
   ss << "\t" << "uri.get_host() = \"" << get_host() << "\"" << std::endl;
 
-  if(!port)
-    ss << "\t" << "uri.get_port() = unset optional<uint16_t>" << std::endl;
+  if(_port == -1)
+    ss << "\t" << "uri.get_port() = invalid (uninitialized)" << std::endl;
   else
-    ss << "\t" << "uri.get_port() = \"" << port.value() << "\"" << std::endl;
+    ss << "\t" << "uri.get_port() = \"" << _port << "\"" << std::endl;
 
   ss << "\t" << "uri.get_path() = \"" << get_path() << "\"" << std::endl;
   ss << "\t" << "uri.get_fragment() = \"" << get_fragment() << "\"" << std::endl;
 
 
-  std::vector<std::pair<std::string, std::string> > elems = get_query_elements();
+  std::vector<Query> query_elems = get_query_elements();
 
-  if(elems.size() > 0)
+  if(query_elems.size() > 0)
     ss << "\t" << "Query elements:" << std::endl;
 
-  for(auto qry = elems.begin(); qry != elems.end(); qry++) {
-    ss << "\t\t" << qry->first << " -> " << qry->second << std::endl;
+  for(auto qry = query_elems.begin(); qry != query_elems.end(); qry++) {
+    ss << "\t\t" << qry->key << " -> " << qry->value << std::endl;
   }
 
   return ss.str();
 }
 
-}
+} // end namespace hdfs

+ 0 - 130
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/uri.h

@@ -1,130 +0,0 @@
-/**
- * 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 COMMON_HDFS_URI_H_
-#define COMMON_HDFS_URI_H_
-
-#include <iostream>
-#include <string>
-#include <optional.hpp>
-#include <vector>
-
-namespace hdfs
-{
-
-template <class T>
-using optional = std::experimental::optional<T>;
-
-class URI
-{
-    // These are stored in encoded form
-    std::string scheme;
-    std::string user;
-    std::string pass;
-    std::string host;
-    optional<uint16_t> port;
-    std::vector<std::string> path;
-    std::vector<std::pair<std::string,std::string> > query;
-    std::string fragment;
-
-    template <class T>
-    static T from_encoded(bool encoded_output, const T & input) {return encoded_output ? input : decode(input);}
-
-    template <class T>
-    static T to_encoded(bool encoded_input, const T & input) {return encoded_input ? input : encode(input);}
-
-    bool has_authority() const;
-    std::string build_authority(bool encoded_output) const;
-
-    std::string build_path(bool encoded_output) const;
-    void parse_path(bool input_encoded, const std::string &input_path);
-
-public:
-    // Parse a string into a URI.  Returns nullopt if the URI is malformed.
-    static optional<URI> parse_from_string(const std::string &str);
-
-    static std::string encode  (const std::string &input);
-    static std::string decode  (const std::string &input);
-
-    std::string get_scheme(bool encoded_output=false) const
-    { return from_encoded(encoded_output,scheme); }
-
-    void set_scheme(const std::string &s, bool encoded_input=false)
-    { scheme = to_encoded(encoded_input,s); }
-
-    // empty if none.
-    std::string get_host(bool encoded_output=false) const
-    { return from_encoded(encoded_output,host); }
-
-    void set_host(const std::string& h, bool encoded_input=false)
-    { host = to_encoded(encoded_input,h); }
-
-    // -1 if the port is undefined.
-    optional<uint16_t> get_port() const
-    { return port; }
-
-    void set_port(uint16_t p)
-    { port = p; }
-
-    void clear_port()
-    { port = std::experimental::nullopt; }
-
-    std::string get_path(bool encoded_output=false) const;
-
-    std::vector<std::string> get_path_elements(bool encoded_output=false) const;
-
-    void set_path(const std::string &p, bool encoded_input=false) {
-        parse_path(encoded_input, p);
-    }
-
-    void add_path(const std::string &p, bool encoded_input=false);
-
-    std::string get_query(bool encoded_output=false) const;
-
-    std::vector< std::pair<std::string, std::string> > get_query_elements(bool encoded_output=false) const;
-
-    // Not that set_query must always pass in encoded strings
-    void set_query(const std::string &q);
-
-    // Adds a parameter onto the query; does not check if it already exists
-    //   e.g. parseFromString("foo?bar=baz").addQuery("bing","bang")
-    //   would leave "bar=baz&bing=bang" as the query
-    void add_query(const std::string &name, const std::string & value, bool encoded_input=false);
-
-    // Removes the query part if exists
-    //   e.g. parseFromString("foo?bar=baz&bing=bang&bar=bong").removeQueries("bar")
-    //   would leave bing=bang as the query
-    void remove_queries(const std::string &q_name, bool encoded_input=false);
-
-    std::string get_fragment(bool encoded_output=false) const
-    { return from_encoded(encoded_output, fragment); }
-
-    void set_fragment(const std::string &f, bool encoded_input=false)
-    { fragment = to_encoded(encoded_input,f); }
-
-    std::string str(bool encoded_output=true) const;
-
-    // Get a string with each URI field printed on a seperate line
-    std::string GetDebugString() const;
-};
-
-inline std::ostream& operator<<(std::ostream &out, const URI &uri)
-{ return out << uri.str(); }
-
-}
-#endif

+ 5 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc

@@ -221,12 +221,12 @@ void FileSystemImpl::Connect(const std::string &server,
 
     // tmp namenode info just to get this in the right format for BulkResolve
     NamenodeInfo tmp_info;
-    optional<URI> uri = URI::parse_from_string("hdfs://" + cluster_name_);
-    if(!uri) {
+    try {
+      tmp_info.uri = URI::parse_from_string("hdfs://" + cluster_name_);
+    } catch (const uri_parse_error& e) {
       LOG_ERROR(kFileSystem, << "Unable to use URI for cluster " << cluster_name_);
       handler(Status::Error(("Invalid namenode " + cluster_name_ + " in config").c_str()), this);
     }
-    tmp_info.uri = uri.value();
 
     resolved_namenodes = BulkResolve(&io_service_->io_service(), {tmp_info});
   }
@@ -258,11 +258,8 @@ void FileSystemImpl::ConnectToDefaultFs(const std::function<void(const Status &,
     return;
   }
 
-  optional<uint16_t>  port = options_.defaultFS.get_port();
-  if (!port) {
-    port = kDefaultPort;
-  }
-  std::string port_as_string = std::to_string(*port);
+  int16_t port = options_.defaultFS.get_port_or_default(kDefaultPort);
+  std::string port_as_string = std::to_string(port);
 
   Connect(host, port_as_string, handler);
 }

+ 144 - 95
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/uri_test.cc

@@ -16,103 +16,127 @@
  * limitations under the License.
  */
 
-#include "common/uri.h"
+#include "hdfspp/uri.h"
 #include <gmock/gmock.h>
 
 using ::testing::_;
 
 using namespace hdfs;
 
+
+URI expect_uri_throw(const char *uri) {
+  bool threw = false;
+  std::string what_msg;
+  URI val;
+  try {
+    val = URI::parse_from_string(uri);
+  } catch (const uri_parse_error& e) {
+    threw = true;
+    what_msg = e.what();
+  } catch (...) {
+    threw = true;
+  }
+
+  EXPECT_TRUE(threw);
+  EXPECT_EQ(what_msg, uri);
+  return val;
+}
+
+URI expect_uri_nothrow(const char *uri) {
+  bool threw = false;
+  std::string what_msg;
+  URI val;
+  try {
+    val = URI::parse_from_string(uri);
+  } catch (const uri_parse_error& e) {
+    threw = true;
+    what_msg = e.what();
+  } catch (...) {
+    threw = true;
+  }
+
+  EXPECT_FALSE(threw);
+  EXPECT_EQ(what_msg, "");
+  return val;
+}
+
+
 TEST(UriTest, TestDegenerateInputs) {
   /* Empty input */
-  {
-    optional<URI> uri = URI::parse_from_string("");
-    EXPECT_TRUE(uri && "Empty input");
-  }
+  expect_uri_nothrow("");
 
   /* Invalid encoding */
-  {
-    optional<URI> uri = URI::parse_from_string("%%");
-    EXPECT_FALSE(uri && "Bad input");
-  }
+  expect_uri_throw("%%");
 
   /* Invalid port */
-  {
-    optional<URI> uri = URI::parse_from_string("hdfs://nn:foo/");
-    EXPECT_FALSE(uri && "Bad port");
-  }
+  expect_uri_throw("hdfs://nn:foo/");
 
   /* Negative port */
-  {
-    optional<URI> uri = URI::parse_from_string("hdfs://nn:-100/");
-    EXPECT_FALSE(uri && "Negative port");
-  }
+  expect_uri_throw("hdfs://nn:-100/");
 
   /* Empty paths */
-  {
-    optional<URI> uri = URI::parse_from_string("hdfs://////");
-    EXPECT_TRUE(uri && "Empty paths");
-  }
-
+  expect_uri_nothrow("hdfs://////");
 }
 
 
 TEST(UriTest, TestNominalInputs) {
   /* Simple input */
   {
-    optional<URI> uri = URI::parse_from_string("hdfs:///foo");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("hdfs", uri->get_scheme());
-    EXPECT_EQ("", uri->get_host());
-    EXPECT_EQ(0, uri->get_port().value_or(0));
-    EXPECT_EQ("/foo", uri->get_path());
-    EXPECT_EQ("", uri->get_fragment());
-    EXPECT_EQ("", uri->get_query());
+    URI uri = expect_uri_nothrow("hdfs:///foo");
+    EXPECT_EQ("hdfs", uri.get_scheme());
+    EXPECT_EQ("", uri.get_host());
+    EXPECT_FALSE(uri.has_port());
+    EXPECT_EQ(0, uri.get_port_or_default(0));
+    EXPECT_EQ("/foo", uri.get_path());
+    EXPECT_EQ("", uri.get_fragment());
+    EXPECT_EQ("", uri.get_query());
   }
 
   /* With authority */
   {
-    optional<URI> uri = URI::parse_from_string("hdfs://host:100/foo");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("hdfs", uri->get_scheme());
-    EXPECT_EQ("host", uri->get_host());
-    EXPECT_EQ(100, uri->get_port().value_or(0));
-    EXPECT_EQ("/foo", uri->get_path());
-    EXPECT_EQ("", uri->get_fragment());
-    EXPECT_EQ("", uri->get_query());
+    URI uri = expect_uri_nothrow("hdfs://host:100/foo");
+    EXPECT_EQ("hdfs", uri.get_scheme());
+    EXPECT_EQ("host", uri.get_host());
+    EXPECT_TRUE(uri.has_port());
+    EXPECT_EQ(100, uri.get_port());
+    EXPECT_EQ(100, uri.get_port_or_default(0));
+    EXPECT_EQ("/foo", uri.get_path());
+    EXPECT_EQ("", uri.get_fragment());
+    EXPECT_EQ("", uri.get_query());
   }
 
   /* No scheme */
   {
-    optional<URI> uri = URI::parse_from_string("/foo");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("", uri->get_scheme());
-    EXPECT_EQ("", uri->get_host());
-    EXPECT_EQ(0, uri->get_port().value_or(0));
-    EXPECT_EQ("/foo", uri->get_path());
-    EXPECT_EQ("", uri->get_fragment());
-    EXPECT_EQ("", uri->get_query());
+    URI uri = expect_uri_nothrow("/foo");
+    EXPECT_EQ("", uri.get_scheme());
+    EXPECT_EQ("", uri.get_host());
+    EXPECT_FALSE(uri.has_port());
+    EXPECT_EQ(0, uri.get_port_or_default(0));
+    EXPECT_EQ("/foo", uri.get_path());
+    EXPECT_EQ("", uri.get_fragment());
+    EXPECT_EQ("", uri.get_query());
   }
 
   /* All fields */
   {
-    optional<URI> uri = URI::parse_from_string("hdfs://nn:8020/path/to/data?a=b&c=d#fragment");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("hdfs", uri->get_scheme());
-    EXPECT_EQ("nn", uri->get_host());
-    EXPECT_EQ(8020, uri->get_port().value_or(0));
-    EXPECT_EQ("/path/to/data", uri->get_path());
-    EXPECT_EQ("a=b&c=d", uri->get_query());
-    EXPECT_EQ(3, uri->get_path_elements().size());
-    EXPECT_EQ("path", uri->get_path_elements()[0]);
-    EXPECT_EQ("to", uri->get_path_elements()[1]);
-    EXPECT_EQ("data", uri->get_path_elements()[2]);
-    EXPECT_EQ(2, uri->get_query_elements().size());
-    EXPECT_EQ("a", uri->get_query_elements()[0].first);
-    EXPECT_EQ("b", uri->get_query_elements()[0].second);
-    EXPECT_EQ("c", uri->get_query_elements()[1].first);
-    EXPECT_EQ("d", uri->get_query_elements()[1].second);
-    EXPECT_EQ("fragment", uri->get_fragment());
+    URI uri = expect_uri_nothrow("hdfs://nn:8020/path/to/data?a=b&c=d#fragment");
+    EXPECT_EQ("hdfs", uri.get_scheme());
+    EXPECT_EQ("nn", uri.get_host());
+    EXPECT_TRUE(uri.has_port());
+    EXPECT_EQ(8020, uri.get_port());
+    EXPECT_EQ(8020, uri.get_port_or_default(0));
+    EXPECT_EQ("/path/to/data", uri.get_path());
+    EXPECT_EQ("a=b&c=d", uri.get_query());
+    EXPECT_EQ(3, uri.get_path_elements().size());
+    EXPECT_EQ("path", uri.get_path_elements()[0]);
+    EXPECT_EQ("to", uri.get_path_elements()[1]);
+    EXPECT_EQ("data", uri.get_path_elements()[2]);
+    EXPECT_EQ(2, uri.get_query_elements().size());
+    EXPECT_EQ("a", uri.get_query_elements()[0].key);
+    EXPECT_EQ("b", uri.get_query_elements()[0].value);
+    EXPECT_EQ("c", uri.get_query_elements()[1].key);
+    EXPECT_EQ("d", uri.get_query_elements()[1].value);
+    EXPECT_EQ("fragment", uri.get_fragment());
   }
 }
 
@@ -121,52 +145,48 @@ TEST(UriTest, TestEncodedInputs) {
 
   /* Encoded input */
   {
-    optional<URI> uri = URI::parse_from_string("S://%5E:1/+%5E%20?%5E=%5E#%5E");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("S", uri->get_scheme());
-    EXPECT_EQ("^", uri->get_host());
-    EXPECT_EQ(1, uri->get_port().value_or(0));
-    EXPECT_EQ("/ ^ ", uri->get_path());
-    EXPECT_EQ("^", uri->get_fragment());
-    EXPECT_EQ("^=^", uri->get_query());
+    URI uri = expect_uri_nothrow("S://%5E:1/+%5E%20?%5E=%5E#%5E");
+    EXPECT_EQ("S", uri.get_scheme());
+    EXPECT_EQ("^", uri.get_host());
+    EXPECT_EQ(1, uri.get_port_or_default(0));
+    EXPECT_EQ("/ ^ ", uri.get_path());
+    EXPECT_EQ("^", uri.get_fragment());
+    EXPECT_EQ("^=^", uri.get_query());
   }
 
   /* Lowercase */
   {
-    optional<URI> uri = URI::parse_from_string("S://%5e:1/+%5e%20?%5e=%5e#%5e");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("S", uri->get_scheme());
-    EXPECT_EQ("^", uri->get_host());
-    EXPECT_EQ(1, uri->get_port().value_or(0));
-    EXPECT_EQ("/ ^ ", uri->get_path());
-    EXPECT_EQ("^", uri->get_fragment());
-    EXPECT_EQ("^=^", uri->get_query());
+    URI uri = expect_uri_nothrow("S://%5e:1/+%5e%20?%5e=%5e#%5e");
+    EXPECT_EQ("S", uri.get_scheme());
+    EXPECT_EQ("^", uri.get_host());
+    EXPECT_EQ(1, uri.get_port_or_default(0));
+    EXPECT_EQ("/ ^ ", uri.get_path());
+    EXPECT_EQ("^", uri.get_fragment());
+    EXPECT_EQ("^=^", uri.get_query());
   }
 }
 
 TEST(UriTest, TestDecodedInputsAndOutputs) {
   /* All fields non-encoded and shouldn't be interpreted */
   {
-    optional<URI> uri = URI::parse_from_string("S://%25/%25+?%25=%25#%25");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("S", uri->get_scheme());
-    EXPECT_EQ("%", uri->get_host());
-    EXPECT_EQ(0, uri->get_port().value_or(0));
-    EXPECT_EQ("/% ", uri->get_path());
-    EXPECT_EQ("%", uri->get_fragment());
-    EXPECT_EQ("%=%", uri->get_query());
+    URI uri = expect_uri_nothrow("S://%25/%25+?%25=%25#%25");
+    EXPECT_EQ("S", uri.get_scheme());
+    EXPECT_EQ("%", uri.get_host());
+    EXPECT_EQ(0, uri.get_port_or_default(0));
+    EXPECT_EQ("/% ", uri.get_path());
+    EXPECT_EQ("%", uri.get_fragment());
+    EXPECT_EQ("%=%", uri.get_query());
   }
 
   /* All fields encode fields on their way out */
   {
-    optional<URI> uri = URI::parse_from_string("S://%25/%25+?%25=%25#%25");
-    ASSERT_TRUE(uri && "Parsed");
-    EXPECT_EQ("S", uri->get_scheme(true));
-    EXPECT_EQ("%25", uri->get_host(true));
-    EXPECT_EQ(0, uri->get_port().value_or(0));
-    EXPECT_EQ("/%25+", uri->get_path(true));
-    EXPECT_EQ("%25", uri->get_fragment(true));
-    EXPECT_EQ("%25=%25", uri->get_query(true));
+    URI uri = expect_uri_nothrow("S://%25/%25+?%25=%25#%25");
+    EXPECT_EQ("S", uri.get_scheme(true));
+    EXPECT_EQ("%25", uri.get_host(true));
+    EXPECT_EQ(0, uri.get_port_or_default(0));
+    EXPECT_EQ("/%25+", uri.get_path(true));
+    EXPECT_EQ("%25", uri.get_fragment(true));
+    EXPECT_EQ("%25=%25", uri.get_query(true));
   }
 
 }
@@ -235,6 +255,35 @@ TEST(UriTest, TestSetters) {
 
 }
 
+TEST(UriTest, QueryManip) {
+  // Not encoded, just basic adding and removing query parts
+  {
+    URI uri = URI::parse_from_string("hdfs://nn:8020/path?thedude=lebowski&donny=outofhiselement");
+    EXPECT_TRUE(uri.has_port());
+    EXPECT_EQ(uri.get_query(), "thedude=lebowski&donny=outofhiselement");
+
+    std::vector<URI::Query> queries = uri.get_query_elements();
+    EXPECT_EQ(queries.size(), 2);
+    EXPECT_EQ(queries[0].key, "thedude");
+    EXPECT_EQ(queries[0].value, "lebowski");
+    EXPECT_EQ(queries[1].key, "donny");
+    EXPECT_EQ(queries[1].value, "outofhiselement");
+
+    uri.remove_query("donny"); // that's a bummer, man
+    EXPECT_EQ(uri.get_query(), "thedude=lebowski");
+    queries = uri.get_query_elements();
+    EXPECT_EQ(queries.size(), 1);
+    EXPECT_EQ(queries[0].key, "thedude");
+    EXPECT_EQ(queries[0].value, "lebowski");
+
+    uri.add_query("HeyPeter", "CheckItOut");
+    EXPECT_EQ(uri.get_query(), "thedude=lebowski&HeyPeter=CheckItOut");
+    queries = uri.get_query_elements();
+    EXPECT_EQ(queries.size(), 2);
+  }
+
+}
+
 int main(int argc, char *argv[]) {
   /*
    *  The following line must be executed to initialize Google Mock

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_allowSnapshot.cc

@@ -66,19 +66,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->AllowSnapshot(uri->get_path());
+  hdfs::Status status = fs->AllowSnapshot(uri.get_path());
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_cat.cc

@@ -67,19 +67,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  readFile(fs, uri->get_path(), 0, stdout, false);
+  readFile(fs, uri.get_path(), 0, stdout, false);
 
   // Clean up static data and prevent valgrind memory leaks
   google::protobuf::ShutdownProtobufLibrary();

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chgrp.cc

@@ -101,13 +101,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind + 1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -121,7 +117,7 @@ int main(int argc, char *argv[]) {
   };
 
   if(!recursive){
-    fs->SetOwner(uri->get_path(), owner, group, handler);
+    fs->SetOwner(uri.get_path(), owner, group, handler);
   }
   else {
     //Allocating shared state, which includes:
@@ -173,7 +169,7 @@ int main(int argc, char *argv[]) {
     };
 
     //Asynchronous call to Find
-    fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
+    fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
   }
 
   /* block until promise is set */

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chmod.cc

@@ -97,13 +97,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind + 1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -119,7 +115,7 @@ int main(int argc, char *argv[]) {
   //strtol() is reading the value with base 8, NULL because we are reading in just one value.
   uint16_t perm = strtol(permissions.c_str(), NULL, 8);
   if(!recursive){
-    fs->SetPermission(uri->get_path(), perm, handler);
+    fs->SetPermission(uri.get_path(), perm, handler);
   }
   else {
     //Allocating shared state, which includes:
@@ -171,7 +167,7 @@ int main(int argc, char *argv[]) {
     };
 
     //Asynchronous call to Find
-    fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
+    fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
   }
 
   /* block until promise is set */

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_chown.cc

@@ -111,13 +111,9 @@ int main(int argc, char *argv[]) {
   }
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -131,7 +127,7 @@ int main(int argc, char *argv[]) {
   };
 
   if(!recursive){
-    fs->SetOwner(uri->get_path(), owner, group, handler);
+    fs->SetOwner(uri.get_path(), owner, group, handler);
   }
   else {
     //Allocating shared state, which includes:
@@ -183,7 +179,7 @@ int main(int argc, char *argv[]) {
     };
 
     //Asynchronous call to Find
-    fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
+    fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
   }
 
   /* block until promise is set */

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_copyToLocal.cc

@@ -66,13 +66,9 @@ int main(int argc, char *argv[]) {
   std::string dest = argv[optind+1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -83,7 +79,7 @@ int main(int argc, char *argv[]) {
     std::cerr << "Unable to open the destination file: " << dest << std::endl;
     exit(EXIT_FAILURE);
   }
-  readFile(fs, uri->get_path(), 0, dst_file, false);
+  readFile(fs, uri.get_path(), 0, dst_file, false);
   std::fclose(dst_file);
 
   // Clean up static data and prevent valgrind memory leaks

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_count.cc

@@ -71,20 +71,16 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
   hdfs::ContentSummary content_summary;
-  hdfs::Status status = fs->GetContentSummary(uri->get_path(), content_summary);
+  hdfs::Status status = fs->GetContentSummary(uri.get_path(), content_summary);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_createSnapshot.cc

@@ -75,19 +75,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->CreateSnapshot(uri->get_path(), name);
+  hdfs::Status status = fs->CreateSnapshot(uri.get_path(), name);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_deleteSnapshot.cc

@@ -67,19 +67,15 @@ int main(int argc, char *argv[]) {
   std::string name = argv[optind+1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->DeleteSnapshot(uri->get_path(), name);
+  hdfs::Status status = fs->DeleteSnapshot(uri.get_path(), name);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_df.cc

@@ -66,13 +66,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_disallowSnapshot.cc

@@ -66,19 +66,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->DisallowSnapshot(uri->get_path());
+  hdfs::Status status = fs->DisallowSnapshot(uri.get_path());
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_du.cc

@@ -92,13 +92,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -161,10 +157,10 @@ int main(int argc, char *argv[]) {
 
   if(!recursive){
     //Asynchronous call to Find
-    fs->GetListing(uri->get_path(), handlerFind);
+    fs->GetListing(uri.get_path(), handlerFind);
   } else {
     //Asynchronous call to Find
-    fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
+    fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
   }
 
   /* block until promise is set */

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc

@@ -92,13 +92,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -136,7 +132,7 @@ int main(int argc, char *argv[]) {
   };
 
   //Asynchronous call to Find
-  fs->Find(uri->get_path(), name, max_depth, handler);
+  fs->Find(uri.get_path(), name, max_depth, handler);
 
   //block until promise is set
   future.get();

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc

@@ -66,13 +66,9 @@ int main(int argc, char *argv[]) {
   std::string dest = argv[optind+1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -83,7 +79,7 @@ int main(int argc, char *argv[]) {
     std::cerr << "Unable to open the destination file: " << dest << std::endl;
     exit(EXIT_FAILURE);
   }
-  readFile(fs, uri->get_path(), 0, dst_file, false);
+  readFile(fs, uri.get_path(), 0, dst_file, false);
   std::fclose(dst_file);
 
   // Clean up static data and prevent valgrind memory leaks

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc

@@ -71,13 +71,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -116,10 +112,10 @@ int main(int argc, char *argv[]) {
 
   if(!recursive){
     //Asynchronous call to GetListing
-    fs->GetListing(uri->get_path(), handler);
+    fs->GetListing(uri.get_path(), handler);
   } else {
     //Asynchronous call to Find
-    fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handler);
+    fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handler);
   }
 
   //block until promise is set

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_mkdir.cc

@@ -78,19 +78,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->Mkdirs(uri->get_path(), permissions, create_parents);
+  hdfs::Status status = fs->Mkdirs(uri.get_path(), permissions, create_parents);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_moveToLocal.cc

@@ -68,13 +68,9 @@ int main(int argc, char *argv[]) {
   std::string dest = argv[optind+1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -85,7 +81,7 @@ int main(int argc, char *argv[]) {
     std::cerr << "Unable to open the destination file: " << dest << std::endl;
     exit(EXIT_FAILURE);
   }
-  readFile(fs, uri->get_path(), 0, dst_file, true);
+  readFile(fs, uri.get_path(), 0, dst_file, true);
   std::fclose(dst_file);
 
   // Clean up static data and prevent valgrind memory leaks

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_renameSnapshot.cc

@@ -68,19 +68,15 @@ int main(int argc, char *argv[]) {
   std::string new_name = argv[optind+2];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->RenameSnapshot(uri->get_path(), old_name, new_name);
+  hdfs::Status status = fs->RenameSnapshot(uri.get_path(), old_name, new_name);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_rm.cc

@@ -70,19 +70,15 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
-  hdfs::Status status = fs->Delete(uri->get_path(), recursive);
+  hdfs::Status status = fs->Delete(uri.get_path(), recursive);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc

@@ -90,13 +90,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind + 1];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), true);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, true);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -161,7 +157,7 @@ int main(int argc, char *argv[]) {
   };
 
   //Asynchronous call to Find
-  fs->Find(uri->get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
+  fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind);
 
   /* block until promise is set */
   hdfs::Status status = future.get();

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc

@@ -65,20 +65,16 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
   }
 
   hdfs::StatInfo stat_info;
-  hdfs::Status status = fs->GetFileInfo(uri->get_path(), stat_info);
+  hdfs::Status status = fs->GetFileInfo(uri.get_path(), stat_info);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);

+ 5 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc

@@ -73,13 +73,9 @@ int main(int argc, char *argv[]) {
   std::string uri_path = argv[optind];
 
   //Building a URI object from the given uri_path
-  hdfs::optional<hdfs::URI> uri = hdfs::URI::parse_from_string(uri_path);
-  if (!uri) {
-    std::cerr << "Malformed URI: " << uri_path << std::endl;
-    exit(EXIT_FAILURE);
-  }
+  hdfs::URI uri = hdfs::parse_path_or_exit(uri_path);
 
-  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri.value(), false);
+  std::shared_ptr<hdfs::FileSystem> fs = hdfs::doConnect(uri, false);
   if (!fs) {
     std::cerr << "Could not connect the file system. " << std::endl;
     exit(EXIT_FAILURE);
@@ -87,7 +83,7 @@ int main(int argc, char *argv[]) {
 
   //We need to get the size of the file using stat
   hdfs::StatInfo stat_info;
-  hdfs::Status status = fs->GetFileInfo(uri->get_path(), stat_info);
+  hdfs::Status status = fs->GetFileInfo(uri.get_path(), stat_info);
   if (!status.ok()) {
     std::cerr << "Error: " << status.ToString() << std::endl;
     exit(EXIT_FAILURE);
@@ -101,7 +97,7 @@ int main(int argc, char *argv[]) {
 
   do {
     off_t current_length = (off_t) stat_info.length;
-    readFile(fs, uri->get_path(), offset, stdout, false);
+    readFile(fs, uri.get_path(), offset, stdout, false);
 
     //Exit if -f flag was not set
     if(!follow){
@@ -112,7 +108,7 @@ int main(int argc, char *argv[]) {
       //Sleep for the REFRESH_RATE
       sleep(REFRESH_RATE);
       //Use stat to check the new filesize.
-      status = fs->GetFileInfo(uri->get_path(), stat_info);
+      status = fs->GetFileInfo(uri.get_path(), stat_info);
       if (!status.ok()) {
         std::cerr << "Error: " << status.ToString() << std::endl;
         exit(EXIT_FAILURE);

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/tools_common.cc

@@ -47,7 +47,7 @@ namespace hdfs {
     //Check if the user supplied the host
     if(!uri.get_host().empty()){
       //If port is supplied we use it, otherwise we use the empty string so that it will be looked up in configs.
-      std::string port = (uri.get_port()) ? std::to_string(uri.get_port().value()) : "";
+      std::string port = uri.has_port() ? std::to_string(uri.get_port()) : "";
       status = fs->Connect(uri.get_host(), port);
       if (!status.ok()) {
         std::cerr << "Could not connect to " << uri.get_host() << ":" << port << ". " << status.ToString() << std::endl;
@@ -112,4 +112,17 @@ namespace hdfs {
     } while (last_bytes_read > 0);
     return;
   }
+
+
+  URI parse_path_or_exit(const std::string& path)
+  {
+    URI uri;
+    try {
+      uri = hdfs::URI::parse_from_string(path);
+    } catch (const uri_parse_error& e) {
+      std::cerr << "Malformed URI: " << path << std::endl;
+      exit(EXIT_FAILURE);
+    }
+    return uri;
+  }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/tools_common.h

@@ -21,6 +21,7 @@
 #define TOOLS_COMMON_H_
 
 #include "hdfspp/hdfspp.h"
+#include "hdfspp/uri.h"
 #include "common/hdfs_configuration.h"
 #include "common/configuration_loader.h"
 #include <mutex>
@@ -32,6 +33,8 @@ namespace hdfs {
 
   //Open HDFS file at offset, read it to destination file, optionally delete source file
   void readFile(std::shared_ptr<hdfs::FileSystem> fs, std::string path, off_t offset, std::FILE* dst_file, bool to_delete);
+
+  URI parse_path_or_exit(const std::string& path);
 }
 
 #endif