You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ab...@apache.org on 2021/04/23 17:13:06 UTC

[nifi-minifi-cpp] branch main updated (016e034 -> 6bc4b90)

This is an automated email from the ASF dual-hosted git repository.

aboda pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git.


    from 016e034  MINIFICPP-1345 Add flake8 check for python files
     new 870249f  MINIFICPP-1032 Refactor parse_url
     new 6bc4b90  MINIFICPP-1547 - Change default c2 protocol

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 C2.md                                              |   8 +-
 bootstrap.sh                                       |   2 +-
 conf/minifi.properties                             |   5 +-
 encrypt-config/tests/resources/minifi.properties   |   6 +-
 ...th-additional-sensitive-props.minifi.properties |   8 +-
 libminifi/include/RemoteProcessorGroupPort.h       |  22 ++-
 libminifi/include/utils/HTTPClient.h               |  24 +++-
 libminifi/src/c2/C2Agent.cpp                       |  24 ++--
 libminifi/src/utils/HTTPClient.cpp                 | 155 ++++++++++++++-------
 .../test/resources/encrypted.minifi.properties     |   6 +-
 libminifi/test/unit/HTTPUtilTests.cpp              |  97 ++++++++-----
 11 files changed, 220 insertions(+), 137 deletions(-)

[nifi-minifi-cpp] 02/02: MINIFICPP-1547 - Change default c2 protocol

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aboda pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit 6bc4b9057ee841e170bf5087bb86ac379e82a0b8
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Wed Apr 21 15:49:30 2021 +0200

    MINIFICPP-1547 - Change default c2 protocol
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #1058
---
 C2.md                                                             | 8 ++++----
 bootstrap.sh                                                      | 2 +-
 conf/minifi.properties                                            | 5 +++--
 encrypt-config/tests/resources/minifi.properties                  | 6 +++---
 .../resources/with-additional-sensitive-props.minifi.properties   | 8 ++++----
 libminifi/src/c2/C2Agent.cpp                                      | 8 ++++----
 libminifi/test/resources/encrypted.minifi.properties              | 6 +++---
 7 files changed, 22 insertions(+), 21 deletions(-)

diff --git a/C2.md b/C2.md
index 560b8ea..864e5202 100644
--- a/C2.md
+++ b/C2.md
@@ -63,11 +63,11 @@ be requested via C2 DESCRIBE manifest command.
 	# specify classes for the AST response
 	nifi.c2.root.classes=DeviceInfoNode,AgentInformation,FlowInformation
 	
-	# specify C2 protocol -- default is CoapProtocol if not specified
-	#nifi.c2.agent.protocol.class=CoapProtocol
-	# may also use MQTT or REST
-	# nifi.c2.agent.protocol.class=MQTTC2Protocol
+	# specify C2 protocol -- default is RESTSender if not specified
 	nifi.c2.agent.protocol.class=RESTSender
+	# may also use MQTT or CoapProtocol
+	# nifi.c2.agent.protocol.class=MQTTC2Protocol
+	# nifi.c2.agent.protocol.class=CoapProtocol
 	
 	# control c2 heartbeat interval in millisecocnds
 	nifi.c2.agent.heartbeat.period=250
diff --git a/bootstrap.sh b/bootstrap.sh
index 2401996..34fa8d1 100755
--- a/bootstrap.sh
+++ b/bootstrap.sh
@@ -292,7 +292,7 @@ add_disabled_option MQTT_ENABLED ${FALSE} "ENABLE_MQTT"
 add_disabled_option PYTHON_ENABLED ${FALSE} "ENABLE_PYTHON"
 add_dependency PYTHON_ENABLED "python"
 
-add_disabled_option COAP_ENABLED ${TRUE} "ENABLE_COAP"
+add_disabled_option COAP_ENABLED ${FALSE} "ENABLE_COAP"
 add_dependency COAP_ENABLED "automake"
 add_dependency COAP_ENABLED "autoconf"
 add_dependency COAP_ENABLED "libtool"
diff --git a/conf/minifi.properties b/conf/minifi.properties
index e5f2a1a..66dfbe9 100644
--- a/conf/minifi.properties
+++ b/conf/minifi.properties
@@ -65,8 +65,9 @@ nifi.content.repository.class.name=DatabaseContentRepository
 ## define those with missing options
 #nifi.c2.enable=true
 ## define protocol parameters
-## The default is CoAP, if that extension is built. 
-## Alternatively, you may use RESTSender if http-curl is built
+## The default is RESTSender.
+## Alternatively, you may use CoapProtocol if that extension is built.
+#nifi.c2.agent.protocol.class=RESTSender
 #nifi.c2.agent.protocol.class=CoapProtocol
 #nifi.c2.agent.coap.host=
 #nifi.c2.agent.coap.port=
diff --git a/encrypt-config/tests/resources/minifi.properties b/encrypt-config/tests/resources/minifi.properties
index f1ca0a2..1cf2549 100644
--- a/encrypt-config/tests/resources/minifi.properties
+++ b/encrypt-config/tests/resources/minifi.properties
@@ -52,10 +52,10 @@ nifi.rest.api.password=password
 ## define those with missing options
 nifi.c2.enable=true
 ## define protocol parameters
-## The default is CoAP, if that extension is built.
-## Alternatively, you may use RESTSender if http-curl is built
-#nifi.c2.agent.protocol.class=CoapProtocol
+## The default is RESTSender.
+## Alternatively, you may use CoapProtocol if that extension is built.
 nifi.c2.agent.protocol.class=RESTSender
+#nifi.c2.agent.protocol.class=CoapProtocol
 #nifi.c2.agent.coap.host=
 #nifi.c2.agent.coap.port=
 ## base URL of the c2 server,
diff --git a/encrypt-config/tests/resources/with-additional-sensitive-props.minifi.properties b/encrypt-config/tests/resources/with-additional-sensitive-props.minifi.properties
index d021f0f..2ca29b9 100644
--- a/encrypt-config/tests/resources/with-additional-sensitive-props.minifi.properties
+++ b/encrypt-config/tests/resources/with-additional-sensitive-props.minifi.properties
@@ -29,7 +29,7 @@ nifi.database.content.repository.directory.default=${MINIFI_HOME}/content_reposi
 
 nifi.remote.input.secure=true
 nifi.security.need.ClientAuth=
-nifi.security.client.certificate= 	
+nifi.security.client.certificate=
 nifi.security.client.private.key=
 nifi.security.client.pass.phrase=correct_horse_battery_staple
 nifi.security.client.ca.certificate=
@@ -54,10 +54,10 @@ nifi.rest.api.password=password
 ## define those with missing options
 nifi.c2.enable=true
 ## define protocol parameters
-## The default is CoAP, if that extension is built.
-## Alternatively, you may use RESTSender if http-curl is built
-#nifi.c2.agent.protocol.class=CoapProtocol
+## The default is RESTSender.
+## Alternatively, you may use CoapProtocol if that extension is built.
 nifi.c2.agent.protocol.class=RESTSender
+#nifi.c2.agent.protocol.class=CoapProtocol
 #nifi.c2.agent.coap.host=
 #nifi.c2.agent.coap.port=
 ## base URL of the c2 server,
diff --git a/libminifi/src/c2/C2Agent.cpp b/libminifi/src/c2/C2Agent.cpp
index 9c914cc..8d962b5 100644
--- a/libminifi/src/c2/C2Agent.cpp
+++ b/libminifi/src/c2/C2Agent.cpp
@@ -141,21 +141,21 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
 
   if (!reconfigure) {
     if (!configure->get("nifi.c2.agent.protocol.class", "c2.agent.protocol.class", clazz)) {
-      clazz = "CoapProtocol";
+      clazz = "RESTSender";
     }
     logger_->log_info("Class is %s", clazz);
 
     auto protocol = core::ClassLoader::getDefaultClassLoader().instantiateRaw(clazz, clazz);
     if (protocol == nullptr) {
       logger_->log_warn("Class %s not found", clazz);
-      protocol = core::ClassLoader::getDefaultClassLoader().instantiateRaw("CoapProtocol", "CoapProtocol");
+      protocol = core::ClassLoader::getDefaultClassLoader().instantiateRaw("RESTSender", "RESTSender");
       if (!protocol) {
-        const char* errmsg = "Attempted to load CoapProtocol. To enable C2, please specify an active protocol for this agent.";
+        const char* errmsg = "Attempted to load RESTSender. To enable C2, please specify an active protocol for this agent.";
         logger_->log_error(errmsg);
         throw minifi::Exception{ minifi::GENERAL_EXCEPTION, errmsg };
       }
 
-      logger_->log_info("Class is CoapProtocol");
+      logger_->log_info("Class is RESTSender");
     }
 
     // Since !reconfigure, the call comes from the ctor and protocol_ is null, therefore no delete is necessary
diff --git a/libminifi/test/resources/encrypted.minifi.properties b/libminifi/test/resources/encrypted.minifi.properties
index 6ddb95f..c37a5a9 100644
--- a/libminifi/test/resources/encrypted.minifi.properties
+++ b/libminifi/test/resources/encrypted.minifi.properties
@@ -54,10 +54,10 @@ nifi.rest.api.password.protected=xsalsa20poly1305
 ## define those with missing options
 nifi.c2.enable=true
 ## define protocol parameters
-## The default is CoAP, if that extension is built.
-## Alternatively, you may use RESTSender if http-curl is built
-#nifi.c2.agent.protocol.class=CoapProtocol
+## The default is RESTSender.
+## Alternatively, you may use CoapProtocol if that extension is built.
 nifi.c2.agent.protocol.class=RESTSender
+#nifi.c2.agent.protocol.class=CoapProtocol
 #nifi.c2.agent.coap.host=
 #nifi.c2.agent.coap.port=
 ## base URL of the c2 server,

[nifi-minifi-cpp] 01/02: MINIFICPP-1032 Refactor parse_url

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

aboda pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit 870249f713219cce57f0719efa3e14c6e25e15bf
Author: Ferenc Gerlits <fg...@gmail.com>
AuthorDate: Tue Apr 6 12:51:29 2021 +0200

    MINIFICPP-1032 Refactor parse_url
    
    Modernize the parse_url() function and add error checking.
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #1046
---
 libminifi/include/RemoteProcessorGroupPort.h |  22 ++--
 libminifi/include/utils/HTTPClient.h         |  24 ++++-
 libminifi/src/c2/C2Agent.cpp                 |  16 ++-
 libminifi/src/utils/HTTPClient.cpp           | 155 ++++++++++++++++++---------
 libminifi/test/unit/HTTPUtilTests.cpp        |  97 ++++++++++-------
 5 files changed, 198 insertions(+), 116 deletions(-)

diff --git a/libminifi/include/RemoteProcessorGroupPort.h b/libminifi/include/RemoteProcessorGroupPort.h
index 3c33706..8598103 100644
--- a/libminifi/include/RemoteProcessorGroupPort.h
+++ b/libminifi/include/RemoteProcessorGroupPort.h
@@ -146,23 +146,17 @@ class RemoteProcessorGroupPort : public core::Processor {
    */
   void setURL(std::string val) {
     auto urls = utils::StringUtils::split(val, ",");
-    for (auto url : urls) {
-      logger_->log_trace("Parsing %s", url);
-      std::string host, protocol;
-      int port = -1;
-      url = utils::StringUtils::trim(url);
-      utils::parse_url(&url, &host, &port, &protocol);
-      logger_->log_trace("Parsed -%s- %s %s, %d", url, protocol, host, port);
-      if (port == -1) {
-        if (protocol.find("https") != std::string::npos) {
-          port = 443;
-        } else if (protocol.find("http") != std::string::npos) {
-          port = 80;
-        }
+    for (const auto& url : urls) {
+      utils::URL parsed_url{utils::StringUtils::trim(url)};
+      if (parsed_url.isValid()) {
+        logger_->log_debug("Parsed RPG URL '%s' -> '%s'", url, parsed_url.hostPort());
+        nifi_instances_.push_back({parsed_url.host(), parsed_url.port(), parsed_url.protocol()});
+      } else {
+        logger_->log_error("Could not parse RPG URL '%s'", url);
       }
-      nifi_instances_.push_back({ host, port, protocol });
     }
   }
+
   void setHTTPProxy(const utils::HTTPProxy &proxy) {
     this->proxy_ = proxy;
   }
diff --git a/libminifi/include/utils/HTTPClient.h b/libminifi/include/utils/HTTPClient.h
index 82aa55b..425c716 100644
--- a/libminifi/include/utils/HTTPClient.h
+++ b/libminifi/include/utils/HTTPClient.h
@@ -36,6 +36,7 @@
 #include "controllers/SSLContextService.h"
 #include "core/Deprecated.h"
 #include "utils/gsl.h"
+#include "utils/OptionalUtils.h"
 
 namespace org {
 namespace apache {
@@ -362,10 +363,27 @@ class BaseHTTPClient {
   virtual inline bool matches(const std::string &value, const std::string &sregex) = 0;
 };
 
-extern std::string get_token(utils::BaseHTTPClient *client, std::string username, std::string password);
+std::string get_token(utils::BaseHTTPClient *client, std::string username, std::string password);
+
+class URL {
+ public:
+  explicit URL(const std::string& url_input);
+  bool isValid() const { return is_valid_; }
+  std::string protocol() const { return protocol_; }
+  std::string host() const { return host_; }
+  int port() const;
+  std::string hostPort() const;
+  std::string toString() const;
+
+ private:
+  std::string protocol_;
+  std::string host_;
+  utils::optional<int> port_;
+  utils::optional<std::string> path_;
+  bool is_valid_ = false;
+  std::shared_ptr<logging::Logger> logger_ = logging::LoggerFactory<URL>::getLogger();
+};
 
-extern void parse_url(const std::string *url, std::string *host, int *port, std::string *protocol);
-extern void parse_url(const std::string *url, std::string *host, int *port, std::string *protocol, std::string *path, std::string *query);
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
diff --git a/libminifi/src/c2/C2Agent.cpp b/libminifi/src/c2/C2Agent.cpp
index 0914ed5..9c914cc 100644
--- a/libminifi/src/c2/C2Agent.cpp
+++ b/libminifi/src/c2/C2Agent.cpp
@@ -760,22 +760,20 @@ utils::optional<std::string> C2Agent::fetchFlow(const std::string& uri) const {
     std::stringstream adjusted_url;
     std::string base;
     if (configuration_->get(minifi::Configure::nifi_c2_flow_base_url, base)) {
+      base = utils::StringUtils::trim(base);
       adjusted_url << base;
       if (!utils::StringUtils::endsWith(base, "/")) {
         adjusted_url << "/";
       }
       adjusted_url << uri;
       resolved_url = adjusted_url.str();
-    } else if (configuration_->get("c2.rest.url", base)) {
-      std::string host, protocol;
-      int port = -1;
-      utils::parse_url(&base, &host, &port, &protocol);
-      adjusted_url << protocol << host;
-      if (port > 0) {
-        adjusted_url << ":" << port;
+    } else if (configuration_->get("nifi.c2.rest.url", "c2.rest.url", base)) {
+      utils::URL base_url{utils::StringUtils::trim(base)};
+      if (!base_url.isValid()) {
+        logger_->log_error("Could not parse C2 REST URL '%s'", base);
+        return {};
       }
-      adjusted_url << "/c2/api/" << uri;
-      resolved_url = adjusted_url.str();
+      resolved_url = base_url.hostPort() + "/c2/api/" + uri;
     }
   }
 
diff --git a/libminifi/src/utils/HTTPClient.cpp b/libminifi/src/utils/HTTPClient.cpp
index 3328aa5..fe9d702 100644
--- a/libminifi/src/utils/HTTPClient.cpp
+++ b/libminifi/src/utils/HTTPClient.cpp
@@ -15,8 +15,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "utils/HTTPClient.h"
+#include <algorithm>
 #include <string>
+
+#include "utils/HTTPClient.h"
+#include "utils/StringUtils.h"
+
+namespace {
+
+constexpr const char* HTTP = "http://";
+constexpr const char* HTTPS = "https://";
+
+utils::optional<std::string> parseProtocol(const std::string& url_input) {
+  if (utils::StringUtils::startsWith(url_input, HTTP)) {
+    return HTTP;
+  } else if (utils::StringUtils::startsWith(url_input, HTTPS)) {
+    return HTTPS;
+  } else {
+    return {};
+  }
+}
+
+utils::optional<int> parsePortNumber(const std::string& port_string) {
+  try {
+    size_t pos;
+    int port = std::stoi(port_string, &pos);
+    if (pos == port_string.size()) {
+      return port;
+    }
+  } catch (const std::invalid_argument&) {
+  } catch (const std::out_of_range&) {
+  }
+
+  return {};
+}
+
+}  // namespace
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -48,72 +83,86 @@ std::string get_token(utils::BaseHTTPClient *client, std::string username, std::
   return token;
 }
 
-void parse_url(const std::string *url, std::string *host, int *port, std::string *protocol) {
-  static std::string http("http://");
-  static std::string https("https://");
-
-  if (url->compare(0, http.size(), http) == 0)
-    *protocol = http;
-
-  if (url->compare(0, https.size(), https) == 0)
-    *protocol = https;
-
-  if (!protocol->empty()) {
-    size_t pos = url->find_first_of(":", protocol->size());
+URL::URL(const std::string& url_input) {
+  const auto protocol = parseProtocol(url_input);
+  if (protocol) {
+    protocol_ = *protocol;
+  } else {
+    logger_->log_error("Unknown protocol in URL '%s'", url_input);
+    return;
+  }
 
-    if (pos == std::string::npos) {
-      pos = url->size();
-    }
+  std::string::const_iterator current_pos = url_input.begin();
+  std::advance(current_pos, protocol_.size());
 
-    *host = url->substr(protocol->size(), pos - protocol->size());
-
-    if (pos < url->size() && (*url)[pos] == ':') {
-      size_t ppos = url->find_first_of("/", pos);
-      if (ppos == std::string::npos) {
-        ppos = url->size();
-      }
-      std::string portStr(url->substr(pos + 1, ppos - pos - 1));
-      if (portStr.size() > 0) {
-        *port = std::stoi(portStr);
-      }
+  constexpr const char HOST_TERMINATORS[] = ":/?#";
+  std::string::const_iterator end_of_host = std::find_first_of(current_pos, url_input.end(), std::begin(HOST_TERMINATORS), std::end(HOST_TERMINATORS));
+  host_ = std::string{current_pos, end_of_host};
+  if (host_.empty()) {
+    logger_->log_error("No host found in URL '%s'", url_input);
+    return;
+  }
+  current_pos = end_of_host;
+
+  if (current_pos != url_input.end() && *current_pos == ':') {
+    constexpr const char PORT_TERMINATORS[] = "/?#";
+    ++current_pos;
+    std::string::const_iterator end_of_port = std::find_first_of(current_pos, url_input.end(), std::begin(PORT_TERMINATORS), std::end(PORT_TERMINATORS));
+    const auto port_number = parsePortNumber(std::string{current_pos, end_of_port});
+    if (port_number) {
+      port_ = *port_number;
     } else {
-      // In case the host contains no port, the first part is needed only
-      // For eg.: nifi.io/nifi
-      size_t ppos = host->find_first_of("/");
-      if (ppos != std::string::npos) {
-        *host = host->substr(0, ppos);
-      }
+      logger_->log_error("Could not parse the port number in URL '%s'", url_input);
+      return;
     }
+    current_pos = end_of_port;
+  }
+
+  if (current_pos != url_input.end()) {
+    path_ = std::string{current_pos, url_input.end()};
   }
-}
 
-void parse_url(const std::string *url, std::string *host, int *port, std::string *protocol, std::string *path, std::string *query) {
-  int temp_port = -1;
+  is_valid_ = true;
+}
 
-  parse_url(url, host, &temp_port, protocol);
+int URL::port() const {
+  if (port_) {
+    return *port_;
+  } else if (protocol_ == HTTP) {
+    return 80;
+  } else if (protocol_ == HTTPS) {
+    return 443;
+  } else {
+    throw std::logic_error{"Undefined port in URL: " + toString()};
+  }
+}
 
-  if (host->empty() || protocol->empty()) {
-    return;
+std::string URL::hostPort() const {
+  if (!isValid()) {
+    return "INVALID";
   }
 
-  size_t base_len = host->size() + protocol->size();
-  if (temp_port != -1) {
-    *port = temp_port;
-    base_len += std::to_string(temp_port).size() + 1;  // +1 for the :
+  if (port_) {
+    return protocol_ + host_ + ':' + std::to_string(*port_);
+  } else {
+    return protocol_ + host_;
   }
+}
 
-  auto query_loc = url->find_first_of("?", base_len);
+std::string URL::toString() const {
+  if (!isValid()) {
+    return "INVALID";
+  }
 
-  if (query_loc < url->size()) {
-    *path = url->substr(base_len + 1, query_loc - base_len - 1);
-    *query = url->substr(query_loc + 1, url->size() - query_loc - 1);
+  if (path_) {
+    return hostPort() + *path_;
   } else {
-    *path = url->substr(base_len + 1, url->size() - base_len - 1);
+    return hostPort();
   }
 }
 
-} /* namespace utils */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace utils
+}  // namespace minifi
+}  // namespace nifi
+}  // namespace apache
+}  // namespace org
diff --git a/libminifi/test/unit/HTTPUtilTests.cpp b/libminifi/test/unit/HTTPUtilTests.cpp
index e3b7c45..dfb04f4 100644
--- a/libminifi/test/unit/HTTPUtilTests.cpp
+++ b/libminifi/test/unit/HTTPUtilTests.cpp
@@ -17,51 +17,74 @@
  */
 
 #include <string>
-#include <iostream>
 #include "../TestBase.h"
 #include "utils/HTTPClient.h"
 
-TEST_CASE("TestHTTPUtils::simple", "[test parse no port]") {
-  std::string protocol, host;
-  int port = -1;
-  std::string url = "http://nifi.io/nifi";
-  minifi::utils::parse_url(&url, &host, &port, &protocol);
-  REQUIRE(port == -1);
-  REQUIRE(host == "nifi.io");
-  REQUIRE(protocol == "http://");
+TEST_CASE("The URL class can parse various URL strings", "[URL][parsing]") {
+  const auto canParseURL = [](const std::string& url_string) { return utils::URL{url_string}.toString() == url_string; };
+
+  REQUIRE(canParseURL("http://nifi.io"));
+  REQUIRE(canParseURL("http://nifi.io:777"));
+  REQUIRE(canParseURL("http://nifi.io/nifi"));
+  REQUIRE(canParseURL("https://nifi.somewhere.far.away:321/nifi"));
+  REQUIRE(canParseURL("http://nifi.io?what_is_love"));
+  REQUIRE(canParseURL("http://nifi.io:123?what_is_love"));
+  REQUIRE(canParseURL("https://nifi.io/nifi_path?what_is_love"));
+  REQUIRE(canParseURL("http://nifi.io:4321/nifi_path?what_is_love"));
+  REQUIRE(canParseURL("http://nifi.io#anchors_aweigh"));
+  REQUIRE(canParseURL("https://nifi.io:123#anchors_aweigh"));
+  REQUIRE(canParseURL("http://nifi.io/nifi_path#anchors_aweigh"));
+  REQUIRE(canParseURL("https://nifi.io:4321/nifi_path#anchors_aweigh"));
 }
 
-TEST_CASE("TestHTTPUtils::urlwithport", "[test parse with port]") {
-  std::string protocol, host;
-  int port = -1;
-  std::string url = "https://nifi.somewhere.far.away:321/nifi";
-  minifi::utils::parse_url(&url, &host, &port, &protocol);
-  REQUIRE(port == 321);
-  REQUIRE(host == "nifi.somewhere.far.away");
-  REQUIRE(protocol == "https://");
+TEST_CASE("The URL class will fail to parse invalid URL strings", "[URL][parsing]") {
+  const auto failToParseURL = [](const std::string& url_string) { return utils::URL{url_string}.toString() == "INVALID"; };
+
+  REQUIRE(failToParseURL("mailto:santa.claus@north.pole.org"));
+  REQUIRE(failToParseURL("http:nifi.io"));
+  REQUIRE(failToParseURL("http://"));
+  REQUIRE(failToParseURL("http://:123"));
+  REQUIRE(failToParseURL("http://nifi.io:0x50"));
+  REQUIRE(failToParseURL("http://nifi.io:port_number"));
 }
 
-TEST_CASE("TestHTTPUtils::query", "[test parse query without port]") {
-  std::string protocol, host, path, query;
-  int port = -1;
-  std::string url = "https://nifi.io/nifi/path?what";
-  minifi::utils::parse_url(&url, &host, &port, &protocol, &path, &query);
-  REQUIRE(port == -1);
-  REQUIRE(host == "nifi.io");
-  REQUIRE(protocol == "https://");
-  REQUIRE(path == "nifi/path");
-  REQUIRE(query == "what");
+TEST_CASE("The URL class can extract the port from URL strings", "[URL][port]") {
+  REQUIRE(utils::URL{"http://nifi.io"}.port() == 80);
+  REQUIRE(utils::URL{"http://nifi.io/nifi"}.port() == 80);
+  REQUIRE(utils::URL{"https://nifi.io"}.port() == 443);
+  REQUIRE(utils::URL{"https://nifi.io/nifi"}.port() == 443);
+  REQUIRE(utils::URL{"http://nifi.io:123"}.port() == 123);
+  REQUIRE(utils::URL{"http://nifi.io:123/nifi"}.port() == 123);
+  REQUIRE(utils::URL{"https://nifi.io:456"}.port() == 456);
+  REQUIRE(utils::URL{"https://nifi.io:456/nifi"}.port() == 456);
 }
 
-TEST_CASE("TestHTTPUtils::querywithport", "[test parse query with port]") {
-  std::string protocol, host, path, query;
-  int port = -1;
-  std::string url = "http://nifi.io:4321/nifi_path?what_is_love";
-  minifi::utils::parse_url(&url, &host, &port, &protocol, &path, &query);
-  REQUIRE(port == 4321);
-  REQUIRE(host == "nifi.io");
-  REQUIRE(protocol == "http://");
-  REQUIRE(path == "nifi_path");
-  REQUIRE(query == "what_is_love");
+TEST_CASE("The URL class can extract the host", "[URL][host]") {
+  REQUIRE(utils::URL{"http://nifi.io"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:777"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io/nifi"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"https://nifi.somewhere.far.away:321/nifi"}.host() == "nifi.somewhere.far.away");
+  REQUIRE(utils::URL{"http://nifi.io?what_is_love"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:123?what_is_love"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"https://nifi.io/nifi_path?what_is_love"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:4321/nifi_path?what_is_love"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io#anchors_aweigh"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"https://nifi.io:123#anchors_aweigh"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io/nifi_path#anchors_aweigh"}.host() == "nifi.io");
+  REQUIRE(utils::URL{"https://nifi.io:4321/nifi_path#anchors_aweigh"}.host() == "nifi.io");
 }
 
+TEST_CASE("The URL class can extract the host and port", "[URL][hostPort]") {
+  REQUIRE(utils::URL{"http://nifi.io"}.hostPort() == "http://nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:777"}.hostPort() == "http://nifi.io:777");
+  REQUIRE(utils::URL{"http://nifi.io/nifi"}.hostPort() == "http://nifi.io");
+  REQUIRE(utils::URL{"https://nifi.somewhere.far.away:321/nifi"}.hostPort() == "https://nifi.somewhere.far.away:321");
+  REQUIRE(utils::URL{"http://nifi.io?what_is_love"}.hostPort() == "http://nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:123?what_is_love"}.hostPort() == "http://nifi.io:123");
+  REQUIRE(utils::URL{"https://nifi.io/nifi_path?what_is_love"}.hostPort() == "https://nifi.io");
+  REQUIRE(utils::URL{"http://nifi.io:4321/nifi_path?what_is_love"}.hostPort() == "http://nifi.io:4321");
+  REQUIRE(utils::URL{"http://nifi.io#anchors_aweigh"}.hostPort() == "http://nifi.io");
+  REQUIRE(utils::URL{"https://nifi.io:123#anchors_aweigh"}.hostPort() == "https://nifi.io:123");
+  REQUIRE(utils::URL{"http://nifi.io/nifi_path#anchors_aweigh"}.hostPort() == "http://nifi.io");
+  REQUIRE(utils::URL{"https://nifi.io:4321/nifi_path#anchors_aweigh"}.hostPort() == "https://nifi.io:4321");
+}