You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2018/11/14 21:21:08 UTC

nifi-minifi-cpp git commit: MINIFICPP-675: Fix issue with header evaluation and re-enable test

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master 0c58dcf6d -> 1257e5291


MINIFICPP-675: Fix issue with header evaluation and re-enable test

MINIFICPP-668: don't append port if it is not valid

This closes #438.

Signed-off-by: Aldrin Piri <al...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/commit/1257e529
Tree: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/tree/1257e529
Diff: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/diff/1257e529

Branch: refs/heads/master
Commit: 1257e529170bd5b8f857c1e911b6c77b031e0cb6
Parents: 0c58dcf
Author: Marc Parisi <ph...@apache.org>
Authored: Wed Nov 14 14:22:28 2018 -0500
Committer: Aldrin Piri <al...@apache.org>
Committed: Wed Nov 14 16:20:45 2018 -0500

----------------------------------------------------------------------
 extensions/http-curl/client/HTTPClient.h        | 23 +++++++-
 .../http-curl/sitetosite/HTTPProtocol.cpp       | 13 ++---
 extensions/http-curl/tests/CMakeLists.txt       |  1 +
 extensions/http-curl/tests/HTTPHandlers.h       | 31 ++++++++--
 .../http-curl/tests/HTTPIntegrationBase.h       |  3 +-
 .../http-curl/tests/HTTPSiteToSiteTests.cpp     | 61 +++++++++++---------
 libminifi/src/RemoteProcessorGroupPort.cpp      | 26 ++++++---
 libminifi/test/resources/TestHTTPSiteToSite.yml |  2 +-
 8 files changed, 107 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/client/HTTPClient.h
----------------------------------------------------------------------
diff --git a/extensions/http-curl/client/HTTPClient.h b/extensions/http-curl/client/HTTPClient.h
index 32370e5..f83df10 100644
--- a/extensions/http-curl/client/HTTPClient.h
+++ b/extensions/http-curl/client/HTTPClient.h
@@ -131,11 +131,10 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   void setDisableHostVerification() override;
 
-  std::string getURL() const{
+  std::string getURL() const {
     return url_;
   }
 
-
   const std::vector<std::string> &getHeaders() override {
     return header_response_.header_tokens_;
 
@@ -148,6 +147,24 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
   }
 
   /**
+   * Locates the header value ignoring case. This is different than returning a mapping
+   * of all parsed headers.
+   * This function acknowledges that header entries should be searched case insensitively.
+   * @param key key to search
+   * @return header value.
+   */
+  const std::string getHeaderValue(const std::string &key) {
+    std::string ret;
+    for (const auto &kv : header_response_.header_mapping_) {
+      if (utils::StringUtils::equalsIgnoreCase(key, kv.first)) {
+        ret = kv.second;
+        break;
+      }
+    }
+    return ret;
+  }
+
+  /**
    * Determines if we are connected and operating
    */
   virtual bool isRunning() override {
@@ -199,7 +216,7 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
     }
     return CURLE_OK;
 #else
-	  return CURLE_FAILED_INIT;
+    return CURLE_FAILED_INIT;
 #endif
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/sitetosite/HTTPProtocol.cpp
----------------------------------------------------------------------
diff --git a/extensions/http-curl/sitetosite/HTTPProtocol.cpp b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
index c37b4d6..df3bdc8 100644
--- a/extensions/http-curl/sitetosite/HTTPProtocol.cpp
+++ b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
@@ -62,13 +62,12 @@ std::shared_ptr<Transaction> HttpSiteToSiteClient::createTransaction(std::string
   client->setPostFields("");
   client->submit();
   if (peer_->getStream() != nullptr)
-  logger_->log_debug("Closing %s",((io::HttpStream*)peer_->getStream())->getClientRef()->getURL());
+    logger_->log_debug("Closing %s", ((io::HttpStream*) peer_->getStream())->getClientRef()->getURL());
   if (client->getResponseCode() == 201) {
     // parse the headers
-    auto headers = client->getParsedHeaders();
-    auto intent_name = headers["x-location-uri-intent"];
-    if (intent_name == "transaction-url") {
-      auto url = headers["Location"];
+    auto intent_name = client->getHeaderValue("x-location-uri-intent");
+    if (utils::StringUtils::equalsIgnoreCase(intent_name, "transaction-url")) {
+      auto url = client->getHeaderValue("Location");
 
       if (IsNullOrEmpty(&url)) {
         logger_->log_debug("Location is empty");
@@ -286,9 +285,9 @@ void HttpSiteToSiteClient::closeTransaction(const std::string &transactionID) {
 
   logger_->log_debug("Received %d response code from delete", client->getResponseCode());
 
-  if (client->getResponseCode() == 400){
+  if (client->getResponseCode() == 400) {
     std::stringstream message;
-    message << "Received "  << client->getResponseCode() << " from " << uri.str();
+    message << "Received " << client->getResponseCode() << " from " << uri.str();
     throw Exception(SITE2SITE_EXCEPTION, message.str().c_str());
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/CMakeLists.txt b/extensions/http-curl/tests/CMakeLists.txt
index b8d6c69..9b9a537 100644
--- a/extensions/http-curl/tests/CMakeLists.txt
+++ b/extensions/http-curl/tests/CMakeLists.txt
@@ -80,6 +80,7 @@ add_test(NAME HttpPostIntegrationTest COMMAND HttpPostIntegrationTest "${TEST_RE
 add_test(NAME HttpPostIntegrationTestChunked COMMAND HttpPostIntegrationTest "${TEST_RESOURCES}/TestHTTPPostChunkedEncoding.yml" "${TEST_RESOURCES}/")
 add_test(NAME C2VerifyServeResults COMMAND C2VerifyServeResults "${TEST_RESOURCES}/C2VerifyServeResults.yml" "${TEST_RESOURCES}/")
 add_test(NAME C2VerifyHeartbeatAndStop COMMAND C2VerifyHeartbeatAndStop "${TEST_RESOURCES}/C2VerifyHeartbeatAndStop.yml" "${TEST_RESOURCES}/")
+add_test(NAME HTTPSiteToSiteTests COMMAND HTTPSiteToSiteTests "${TEST_RESOURCES}/TestHTTPSiteToSite.yml" "${TEST_RESOURCES}/" "http://localhost:8099/nifi-api")
 add_test(NAME SiteToSiteRestTest COMMAND SiteToSiteRestTest "${TEST_RESOURCES}/TestSite2SiteRest.yml" "${TEST_RESOURCES}/" "http://localhost:8077/nifi-api/site-to-site")
 add_test(NAME ControllerServiceIntegrationTests COMMAND ControllerServiceIntegrationTests "${TEST_RESOURCES}/TestControllerServices.yml" "${TEST_RESOURCES}/")
 add_test(NAME ThreadPoolAdjust COMMAND ThreadPoolAdjust "${TEST_RESOURCES}/ThreadPoolAdjust.yml" "${TEST_RESOURCES}/")

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/tests/HTTPHandlers.h
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/HTTPHandlers.h b/extensions/http-curl/tests/HTTPHandlers.h
index 58bdd3a..704a9a9 100644
--- a/extensions/http-curl/tests/HTTPHandlers.h
+++ b/extensions/http-curl/tests/HTTPHandlers.h
@@ -78,7 +78,28 @@ class PeerResponder : public CivetHandler {
   }
 
   bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::string site2site_rest_resp = "{\"peers\" : [{ \"hostname\": \"localhost\", \"port\": 8082,  \"secure\": false, \"flowFileCount\" : 0 }] }";
+    std::string site2site_rest_resp = "{\"peers\" : [{ \"hostname\": \"localhost\", \"port\": 8099,  \"secure\": false, \"flowFileCount\" : 0 }] }";
+    std::stringstream headers;
+    headers << "HTTP/1.1 200 OK\r\nContent-Type: application/json\r\nContent-Length: " << site2site_rest_resp.length() << "\r\nConnection: close\r\n\r\n";
+    mg_printf(conn, "%s", headers.str().c_str());
+    mg_printf(conn, "%s", site2site_rest_resp.c_str());
+    return true;
+  }
+
+ protected:
+  std::string base_url;
+};
+
+class SiteToSiteBaseResponder : public CivetHandler {
+ public:
+
+  explicit SiteToSiteBaseResponder(const std::string base_url)
+      : base_url(base_url) {
+  }
+
+  bool handleGet(CivetServer *server, struct mg_connection *conn) {
+    std::string site2site_rest_resp =
+        "{\"controller\":{\"id\":\"96dab149-0162-1000-7924-ed3122d6ea2b\",\"name\":\"NiFi Flow\",\"comments\":\"\",\"runningCount\":3,\"stoppedCount\":6,\"invalidCount\":1,\"disabledCount\":0,\"inputPortCount\":1,\"outputPortCount\":1,\"remoteSiteListeningPort\":10443,\"siteToSiteSecure\":false,\"instanceId\":\"13881505-0167-1000-be72-aa29341a3e9a\",\"inputPorts\":[{\"id\":\"471deef6-2a6e-4a7d-912a-81cc17e3a204\",\"name\":\"RPGIN\",\"comments\":\"\",\"state\":\"RUNNING\"}],\"outputPorts\":[{\"id\":\"9cf15a63-0166-1000-1b29-027406d96013\",\"name\":\"ddsga\",\"comments\":\"\",\"state\":\"STOPPED\"}]}}";
     std::stringstream headers;
     headers << "HTTP/1.1 200 OK\r\nContent-Type: application/json\r\nContent-Length: " << site2site_rest_resp.length() << "\r\nConnection: close\r\n\r\n";
     mg_printf(conn, "%s", headers.str().c_str());
@@ -115,7 +136,7 @@ class TransactionResponder : public CivetHandler {
   bool handlePost(CivetServer *server, struct mg_connection *conn) {
     std::string site2site_rest_resp = "";
     std::stringstream headers;
-    headers << "HTTP/1.1 201 OK\r\nContent-Type: application/json\r\nContent-Length: " << site2site_rest_resp.length() << "\r\nx-location-uri-intent: ";
+    headers << "HTTP/1.1 201 OK\r\nContent-Type: application/json\r\nContent-Length: " << site2site_rest_resp.length() << "\r\nX-Location-Uri-Intent: ";
     if (wrong_uri)
       headers << "ohstuff\r\n";
     else
@@ -128,7 +149,7 @@ class TransactionResponder : public CivetHandler {
     else
       port_type = "output-ports";
     if (!empty_transaction_uri)
-      headers << "Location: " << base_url << "/site-to-site/" << port_type << "/" << port_id << "/transactions/" << transaction_id_str << "\r\n";
+      headers << "locAtion: " << base_url << "/site-to-site/" << port_type << "/" << port_id << "/transactions/" << transaction_id_str << "\r\n";
     headers << "Connection: close\r\n\r\n";
     mg_printf(conn, "%s", headers.str().c_str());
     mg_printf(conn, "%s", site2site_rest_resp.c_str());
@@ -176,7 +197,7 @@ class FlowFileResponder : public CivetHandler {
 
     if (!wrong_uri) {
       minifi::io::CivetStream civet_stream(conn);
-      minifi::io::CRCStream<minifi::io::CivetStream> stream(&civet_stream);
+      minifi::io::CRCStream < minifi::io::CivetStream > stream(&civet_stream);
       uint32_t num_attributes;
       uint64_t total_size = 0;
       total_size += stream.read(num_attributes);
@@ -237,7 +258,7 @@ class FlowFileResponder : public CivetHandler {
                 "Connection: close\r\n\r\n",
                 total);
       minifi::io::BaseStream serializer;
-      minifi::io::CRCStream<minifi::io::BaseStream> stream(&serializer);
+      minifi::io::CRCStream < minifi::io::BaseStream > stream(&serializer);
       for (auto flow : flows) {
         uint32_t num_attributes = flow->attributes.size();
         stream.write(num_attributes);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/tests/HTTPIntegrationBase.h
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/HTTPIntegrationBase.h b/extensions/http-curl/tests/HTTPIntegrationBase.h
index 611c11f..8defc56 100644
--- a/extensions/http-curl/tests/HTTPIntegrationBase.h
+++ b/extensions/http-curl/tests/HTTPIntegrationBase.h
@@ -53,7 +53,6 @@ void HTTPIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   parse_http_components(url, port, scheme, path);
   struct mg_callbacks callback;
   if (url.find("localhost") != std::string::npos) {
-
     if (server != nullptr){
       server->addHandler(path,handler);
       return;
@@ -72,4 +71,4 @@ void HTTPIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   }
 }
 
-#endif /* LIBMINIFI_TEST_INTEGRATION_HTTPINTEGRATIONBASE_H_ */
\ No newline at end of file
+#endif /* LIBMINIFI_TEST_INTEGRATION_HTTPINTEGRATIONBASE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp b/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
index 309492e..dd457b6 100644
--- a/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
+++ b/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
@@ -59,10 +59,10 @@ class SiteToSiteTestHarness : public HTTPIntegrationBase {
   }
 
   void testSetup() {
-    LogTestController::getInstance().setDebug<minifi::RemoteProcessorGroupPort>();
-    LogTestController::getInstance().setDebug<minifi::sitetosite::HttpSiteToSiteClient>();
-    LogTestController::getInstance().setDebug<minifi::sitetosite::SiteToSiteClient>();
-    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setTrace<minifi::RemoteProcessorGroupPort>();
+    LogTestController::getInstance().setTrace<minifi::sitetosite::HttpSiteToSiteClient>();
+    LogTestController::getInstance().setTrace<minifi::sitetosite::SiteToSiteClient>();
+    LogTestController::getInstance().setTrace<utils::HTTPClient>();
     LogTestController::getInstance().setTrace<minifi::controllers::SSLContextService>();
     LogTestController::getInstance().setInfo<minifi::FlowController>();
     LogTestController::getInstance().setDebug<core::ConfigurableComponent>();
@@ -75,7 +75,7 @@ class SiteToSiteTestHarness : public HTTPIntegrationBase {
 
     configuration->set("nifi.c2.enable", "false");
     configuration->set("nifi.remote.input.http.enabled", "true");
-    configuration->set("nifi.remote.input.socket.port", "8082");
+    configuration->set("nifi.remote.input.socket.port", "8099");
   }
 
   virtual void waitToVerifyProcessor() {
@@ -123,17 +123,22 @@ struct test_profile {
 void run_variance(std::string test_file_location, bool isSecure, std::string url, const struct test_profile &profile) {
   SiteToSiteTestHarness harness(isSecure);
 
-  SiteToSiteLocationResponder responder(isSecure);
+  SiteToSiteLocationResponder *responder = new SiteToSiteLocationResponder(isSecure);
 
-  TransactionResponder transaction_response(url, "471deef6-2a6e-4a7d-912a-81cc17e3a204", true, profile.transaction_url_broken, profile.empty_transaction_url);
+  TransactionResponder *transaction_response = new TransactionResponder(url, "471deef6-2a6e-4a7d-912a-81cc17e3a204", true, profile.transaction_url_broken, profile.empty_transaction_url);
 
-  std::string transaction_id = transaction_response.getTransactionId();
+  std::string transaction_id = transaction_response->getTransactionId();
 
   harness.setKeyDir("");
 
   std::string controller_loc = url + "/controller";
 
-  harness.setUrl(controller_loc, &responder);
+  std::string basesitetosite = url + "/site-to-site";
+  SiteToSiteBaseResponder *base = new SiteToSiteBaseResponder(basesitetosite);
+
+  harness.setUrl(basesitetosite,base);
+
+  harness.setUrl(controller_loc, responder);
 
   std::string transaction_url = url + "/data-transfer/input-ports/471deef6-2a6e-4a7d-912a-81cc17e3a204/transactions";
   std::string action_url = url + "/site-to-site/input-ports/471deef6-2a6e-4a7d-912a-81cc17e3a204/transactions";
@@ -141,43 +146,43 @@ void run_variance(std::string test_file_location, bool isSecure, std::string url
   std::string transaction_output_url = url + "/data-transfer/output-ports/471deef6-2a6e-4a7d-912a-81cc17e3a203/transactions";
   std::string action_output_url = url + "/site-to-site/output-ports/471deef6-2a6e-4a7d-912a-81cc17e3a203/transactions";
 
-  harness.setUrl(transaction_url, &transaction_response);
+  harness.setUrl(transaction_url, transaction_response);
 
   std::string peer_url = url + "/site-to-site/peers";
 
-  PeerResponder peer_response(url);
+  PeerResponder *peer_response = new PeerResponder(url);
 
-  harness.setUrl(peer_url, &peer_response);
+  harness.setUrl(peer_url, peer_response);
 
   std::string flow_url = action_url + "/" + transaction_id + "/flow-files";
 
-  FlowFileResponder flowResponder(true, profile.flow_url_broken, profile.invalid_checksum);
-  flowResponder.setFlowUrl(flow_url);
-  auto producedFlows = flowResponder.getFlows();
+  FlowFileResponder *flowResponder = new FlowFileResponder(true, profile.flow_url_broken, profile.invalid_checksum);
+  flowResponder->setFlowUrl(flow_url);
+  auto producedFlows = flowResponder->getFlows();
 
-  TransactionResponder transaction_response_output(url, "471deef6-2a6e-4a7d-912a-81cc17e3a203", false, profile.transaction_url_broken, profile.empty_transaction_url);
-  std::string transaction_output_id = transaction_response_output.getTransactionId();
-  transaction_response_output.setFeed(producedFlows);
+  TransactionResponder *transaction_response_output = new TransactionResponder(url, "471deef6-2a6e-4a7d-912a-81cc17e3a203", false, profile.transaction_url_broken, profile.empty_transaction_url);
+  std::string transaction_output_id = transaction_response_output->getTransactionId();
+  transaction_response_output->setFeed(producedFlows);
 
-  harness.setUrl(transaction_output_url, &transaction_response_output);
+  harness.setUrl(transaction_output_url, transaction_response_output);
 
   std::string flow_output_url = action_output_url + "/" + transaction_output_id + "/flow-files";
 
-  FlowFileResponder flowOutputResponder(false, profile.flow_url_broken, profile.invalid_checksum);
-  flowOutputResponder.setFlowUrl(flow_output_url);
-  flowOutputResponder.setFeed(producedFlows);
+  FlowFileResponder* flowOutputResponder = new FlowFileResponder(false, profile.flow_url_broken, profile.invalid_checksum);
+  flowOutputResponder->setFlowUrl(flow_output_url);
+  flowOutputResponder->setFeed(producedFlows);
 
-  harness.setUrl(flow_url, &flowResponder);
-  harness.setUrl(flow_output_url, &flowOutputResponder);
+  harness.setUrl(flow_url, flowResponder);
+  harness.setUrl(flow_output_url, flowOutputResponder);
 
   if (!profile.no_delete) {
     std::string delete_url = transaction_url + "/" + transaction_id;
-    DeleteTransactionResponder deleteResponse(delete_url, "201 OK", 12);
-    harness.setUrl(delete_url, &deleteResponse);
+    DeleteTransactionResponder *deleteResponse = new DeleteTransactionResponder(delete_url, "201 OK", 12);
+    harness.setUrl(delete_url, deleteResponse);
 
     std::string delete_output_url = transaction_output_url + "/" + transaction_output_id;
-    DeleteTransactionResponder deleteOutputResponse(delete_output_url, "201 OK", producedFlows);
-    harness.setUrl(delete_output_url, &deleteOutputResponse);
+    DeleteTransactionResponder *deleteOutputResponse = new DeleteTransactionResponder(delete_output_url, "201 OK", producedFlows);
+    harness.setUrl(delete_output_url, deleteOutputResponse);
   }
 
   harness.run(test_file_location);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/libminifi/src/RemoteProcessorGroupPort.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp
index 7270361..64fff3c 100644
--- a/libminifi/src/RemoteProcessorGroupPort.cpp
+++ b/libminifi/src/RemoteProcessorGroupPort.cpp
@@ -261,7 +261,13 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
     std::string host = nifi.host_;
     std::string protocol = nifi.protocol_;
     int port = nifi.port_;
-    std::string fullUrl = protocol + host + ":" + std::to_string(port) + "/nifi-api/site-to-site";
+    std::stringstream fullUrl;
+    fullUrl << protocol << host;
+    // don't append port if it is 0 ( undefined )
+    if (port > 0) {
+      fullUrl << ":" << std::to_string(port);
+    }
+    fullUrl << "/nifi-api/site-to-site";
 
     configure_->get(Configure::nifi_rest_api_user_name, this->rest_user_name_);
     configure_->get(Configure::nifi_rest_api_password, this->rest_password_);
@@ -269,7 +275,13 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
     std::string token;
     std::unique_ptr<utils::BaseHTTPClient> client = nullptr;
     if (!rest_user_name_.empty()) {
-      std::string loginUrl = protocol + host + ":" + std::to_string(port) + "/nifi-api/access/token";
+      std::stringstream loginUrl;
+      fullUrl << protocol << host;
+      // don't append port if it is 0 ( undefined )
+      if (port > 0) {
+        fullUrl << ":" << std::to_string(port);
+      }
+      fullUrl << "/nifi-api/access/token";
 
       auto client_ptr = core::ClassLoader::getDefaultClassLoader().instantiateRaw("HTTPClient", "HTTPClient");
       if (nullptr == client_ptr) {
@@ -277,10 +289,10 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
         return std::make_pair("", -1);
       }
       client = std::unique_ptr<utils::BaseHTTPClient>(dynamic_cast<utils::BaseHTTPClient*>(client_ptr));
-      client->initialize("GET", loginUrl, ssl_service);
+      client->initialize("GET", loginUrl.str(), ssl_service);
 
       token = utils::get_token(client.get(), this->rest_user_name_, this->rest_password_);
-      logger_->log_debug("Token from NiFi REST Api endpoint %s,  %s", loginUrl, token);
+      logger_->log_debug("Token from NiFi REST Api endpoint %s,  %s", loginUrl.str(), token);
       if (token.empty())
         return std::make_pair("", -1);
     }
@@ -292,7 +304,7 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
     }
     int siteTosite_port_ = -1;
     client = std::unique_ptr<utils::BaseHTTPClient>(dynamic_cast<utils::BaseHTTPClient*>(client_ptr));
-    client->initialize("GET", fullUrl.c_str(), ssl_service);
+    client->initialize("GET", fullUrl.str().c_str(), ssl_service);
     if (!proxy_.host.empty()) {
       client->setHTTPProxy(proxy_);
     }
@@ -330,10 +342,10 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
           return std::make_pair(host, siteTosite_port_);
         }
       } else {
-        logger_->log_error("Cannot output body to content for ProcessGroup::refreshRemoteSite2SiteInfo: received HTTP code %ll from %s", client->getResponseCode(), fullUrl);
+        logger_->log_error("Cannot output body to content for ProcessGroup::refreshRemoteSite2SiteInfo: received HTTP code %ll from %s", client->getResponseCode(), fullUrl.str());
       }
     } else {
-      logger_->log_error("ProcessGroup::refreshRemoteSite2SiteInfo -- curl_easy_perform() failed \n");
+      logger_->log_error("ProcessGroup::refreshRemoteSite2SiteInfo -- curl_easy_perform() failed , response code %d\n", client->getResponseCode());
     }
   }
   return std::make_pair("", -1);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/1257e529/libminifi/test/resources/TestHTTPSiteToSite.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPSiteToSite.yml b/libminifi/test/resources/TestHTTPSiteToSite.yml
index 582f48b..4bb9104 100644
--- a/libminifi/test/resources/TestHTTPSiteToSite.yml
+++ b/libminifi/test/resources/TestHTTPSiteToSite.yml
@@ -67,7 +67,7 @@ Connections:
 Remote Processing Groups:
     - name: NiFi Flow
       id: 471deef6-2a6e-4a7d-912a-81cc17e3a208
-      url: http://localhost:8082/nifi
+      url: http://localhost:8099/nifi
       timeout: 30 secs
       yield period: 1 sec
       Input Ports: