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/08/07 16:53:07 UTC

nifi-minifi-cpp git commit: MINIFICPP-573: Remove lower level build information that is unnecessary

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master 2f43e2bd3 -> e02304b39


MINIFICPP-573: Remove lower level build information that is unnecessary

MINIFICPP-576: Roll system components into bundle. Update tests for these components

This closes #381.

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/e02304b3
Tree: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/tree/e02304b3
Diff: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/diff/e02304b3

Branch: refs/heads/master
Commit: e02304b39258b01feec0f1944830aa3d49a58461
Parents: 2f43e2b
Author: Marc Parisi <ph...@apache.org>
Authored: Tue Jul 24 10:35:08 2018 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Tue Aug 7 12:52:42 2018 -0400

----------------------------------------------------------------------
 .../tests/C2VerifyHeartbeatAndStop.cpp          | 50 ++++++++++++++++++-
 .../http-curl/tests/C2VerifyServeResults.cpp    | 23 +++++++++
 extensions/http-curl/tests/TestServer.h         |  6 +--
 generateVersion.sh                              |  3 +-
 libminifi/include/agent/build_description.h     |  2 +-
 libminifi/include/core/Resource.h               |  2 +-
 .../include/core/state/nodes/AgentInformation.h | 52 --------------------
 .../unit/PropertyValidationAgentInfoTests.cpp   |  6 +--
 8 files changed, 81 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp b/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
index adb2db1..f962198 100644
--- a/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
+++ b/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
@@ -50,6 +50,7 @@
 #include "protocols/RESTReceiver.h"
 #include "protocols/RESTSender.h"
 #include "HTTPIntegrationBase.h"
+#include "agent/build_description.h"
 #include "processors/LogAttribute.h"
 
 class Responder : public CivetHandler {
@@ -57,9 +58,53 @@ class Responder : public CivetHandler {
   explicit Responder(bool isSecure)
       : isSecure(isSecure) {
   }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
+
+    char buffer[blockSize];
+    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
   bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    std::string resp =
-        "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"name\" : \"invoke\"  }, "
+    auto post_data = readPost(conn);
+
+    std::cerr << post_data << std::endl;
+
+    if (!IsNullOrEmpty(post_data)) {
+      rapidjson::Document root;
+      rapidjson::ParseResult ok = root.Parse(post_data.data(), post_data.size());
+      bool found = false;
+      auto operation = root["operation"].GetString();
+      if (operation == "heartbeat") {
+        assert(root.HasMember("agentInfo") == true);
+        assert(root["agentInfo"]["agentManifest"].HasMember("bundles") == true);
+
+        for (auto &bundle : root["agentInfo"]["agentManifest"]["bundles"].GetArray()) {
+          assert(bundle.HasMember("artifact"));
+          std::string str = bundle["artifact"].GetString();
+          if (str == "minifi-system") {
+
+            std::vector<std::string> classes;
+            for (auto &proc : bundle["componentManifest"]["processors"].GetArray()) {
+              classes.push_back(proc["type"].GetString());
+            }
+
+            auto group = minifi::BuildDescription::getClassDescriptions(str);
+            for (auto proc : group.processors_) {
+              assert(std::find(classes.begin(), classes.end(), proc.class_name_) != std::end(classes));
+              found = true;
+            }
+
+          }
+        }
+        assert(found == true);
+      }
+    }
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"name\" : \"invoke\"  }, "
         "{ \"operationid\" : 42, \"operation\" : \"stop\", \"name\" : \"FlowController\"  } ]}";
     mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
               "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
@@ -120,6 +165,7 @@ class VerifyC2Heartbeat : public HTTPIntegrationBase {
     configuration->set("c2.rest.url", "http://localhost:8888/api/heartbeat");
     configuration->set("c2.agent.heartbeat.period", "1000");
     configuration->set("c2.rest.url.ack", "http://localhost:8888/api/heartbeat");
+    configuration->set("nifi.c2.root.classes", "DeviceInfoNode,AgentInformation,FlowInformation");
   }
 
  protected:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/extensions/http-curl/tests/C2VerifyServeResults.cpp
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/C2VerifyServeResults.cpp b/extensions/http-curl/tests/C2VerifyServeResults.cpp
index fbbc8c8..69afb29 100644
--- a/extensions/http-curl/tests/C2VerifyServeResults.cpp
+++ b/extensions/http-curl/tests/C2VerifyServeResults.cpp
@@ -51,6 +51,28 @@
 #include "HTTPIntegrationBase.h"
 #include "processors/LogAttribute.h"
 
+
+class Responder : public CivetHandler {
+ public:
+  explicit Responder(bool isSecure)
+      : isSecure(isSecure) {
+  }
+  bool handlePost(CivetServer *server, struct mg_connection *conn) {
+    std::string resp =
+        "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"name\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"name\" : \"FlowController\"  } ]}";
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+              "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+              resp.length());
+    mg_printf(conn, "%s", resp.c_str());
+    return true;
+  }
+
+ protected:
+  bool isSecure;
+};
+
+
 class VerifyC2Server : public HTTPIntegrationBase {
  public:
   explicit VerifyC2Server(bool isSecure)
@@ -81,6 +103,7 @@ class VerifyC2Server : public HTTPIntegrationBase {
     assert(LogTestController::getInstance().contains("Import offset 0") == true);
 
     assert(LogTestController::getInstance().contains("Outputting success and response") == true);
+
   }
 
   void queryRootProcessGroup(std::shared_ptr<core::ProcessGroup> pg) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/extensions/http-curl/tests/TestServer.h
----------------------------------------------------------------------
diff --git a/extensions/http-curl/tests/TestServer.h b/extensions/http-curl/tests/TestServer.h
index 06f996c..0847270 100644
--- a/extensions/http-curl/tests/TestServer.h
+++ b/extensions/http-curl/tests/TestServer.h
@@ -38,7 +38,7 @@ CivetServer * start_webserver(std::string &port, std::string &rooturi, CivetHand
       "ALL", "ssl_verify_peer", "no", 0 };
 
   std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
+  for (size_t i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
     cpp_options.push_back(options[i]);
   }
   CivetServer *server = new CivetServer(cpp_options);
@@ -53,7 +53,7 @@ CivetServer * start_webserver(std::string &port, std::string &rooturi, CivetHand
   const char *options[] = { "document_root", ".", "listening_ports", port.c_str(), 0 };
 
   std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
+  for (size_t i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
     cpp_options.push_back(options[i]);
   }
   CivetServer *server = new CivetServer(cpp_options);
@@ -77,7 +77,7 @@ bool parse_http_components(const std::string &url, std::string &port, std::strin
   size_t potentialGroups = regex.re_nsub + 1;
   regmatch_t groups[potentialGroups];
   if (regexec(&regex, url.c_str(), potentialGroups, groups, 0) == 0) {
-    for (int i = 0; i < potentialGroups; i++) {
+    for (size_t i = 0; i < potentialGroups; i++) {
       if (groups[i].rm_so == -1)
         break;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/generateVersion.sh
----------------------------------------------------------------------
diff --git a/generateVersion.sh b/generateVersion.sh
index 238104b..d23195a 100755
--- a/generateVersion.sh
+++ b/generateVersion.sh
@@ -64,11 +64,12 @@ EOF
 for EXTENSION in "${extensions_array[@]}"
 do
 cat <<EOF >> "$out_dir/agent_version.h"
-    extensions.push_back("${EXTENSION}");
+      extensions.push_back("${EXTENSION}");
 EOF
 done
 
 cat <<EOF >> "$out_dir/agent_version.h"
+      extensions.push_back("minifi-system");
     }
   	return extensions;
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/libminifi/include/agent/build_description.h
----------------------------------------------------------------------
diff --git a/libminifi/include/agent/build_description.h b/libminifi/include/agent/build_description.h
index b46dde5..aa4b9e0 100644
--- a/libminifi/include/agent/build_description.h
+++ b/libminifi/include/agent/build_description.h
@@ -51,7 +51,7 @@ struct Components {
 class BuildDescription {
  public:
 
-  static struct Components getClassDescriptions(const std::string group = "default") {
+  static struct Components getClassDescriptions(const std::string group = "minifi-system") {
     static std::map<std::string, struct Components> class_mappings;
     if (UNLIKELY(IsNullOrEmpty(class_mappings[group].processors_) && IsNullOrEmpty(class_mappings[group].processors_))) {
       for (auto clazz : core::ClassLoader::getDefaultClassLoader().getClasses(group)) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/libminifi/include/core/Resource.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Resource.h b/libminifi/include/core/Resource.h
index 0dc6723..410b947 100644
--- a/libminifi/include/core/Resource.h
+++ b/libminifi/include/core/Resource.h
@@ -40,7 +40,7 @@ class StaticClassType {
 #ifdef MODULE_NAME
     ClassLoader::getDefaultClassLoader().registerClass(MAKESTRING(MODULE_NAME), name, std::unique_ptr<ObjectFactory>(new DefautObjectFactory<T>()));
 #else
-    ClassLoader::getDefaultClassLoader().registerClass("default", name, std::unique_ptr<ObjectFactory>(new DefautObjectFactory<T>()));
+    ClassLoader::getDefaultClassLoader().registerClass("minifi-system", name, std::unique_ptr<ObjectFactory>(new DefautObjectFactory<T>()));
 #endif
   }
 };

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/libminifi/include/core/state/nodes/AgentInformation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/AgentInformation.h b/libminifi/include/core/state/nodes/AgentInformation.h
index 14e30db..7b72908 100644
--- a/libminifi/include/core/state/nodes/AgentInformation.h
+++ b/libminifi/include/core/state/nodes/AgentInformation.h
@@ -95,21 +95,10 @@ class ComponentManifest : public DeviceInformation {
 
         SerializedResponseNode desc;
         desc.name = group.class_name_;
-
-        SerializedResponseNode bgroup;
-        bgroup.name = "group";
-        bgroup.value = GROUP_STR;
-        SerializedResponseNode artifact;
-        artifact.name = "artifact";
-        artifact.value = group.class_name_;
-
         SerializedResponseNode className;
         className.name = "type";
         className.value = group.class_name_;
 
-        SerializedResponseNode version;
-        version.name = "version";
-        version.value = AgentBuild::VERSION;
 
         if (!group.class_properties_.empty()) {
           SerializedResponseNode props;
@@ -172,42 +161,7 @@ class ComponentManifest : public DeviceInformation {
         dyn_prop.value = group.support_dynamic_;
 
         desc.children.push_back(dyn_prop);
-
-        desc.children.push_back(bgroup);
-        desc.children.push_back(artifact);
         desc.children.push_back(className);
-        desc.children.push_back(version);
-
-        SerializedResponseNode buildInfo;
-        buildInfo.name = "buildInfo";
-
-        SerializedResponseNode build_version;
-        build_version.name = "version";
-        build_version.value = AgentBuild::VERSION;
-
-        SerializedResponseNode build_rev;
-        build_rev.name = "revision";
-        build_rev.value = AgentBuild::BUILD_REV;
-
-        SerializedResponseNode build_date;
-        build_date.name = "timestamp";
-        build_date.value = (uint64_t) std::stoull(AgentBuild::BUILD_DATE);
-
-        SerializedResponseNode compiler_command;
-        compiler_command.name = "compiler";
-        compiler_command.value = AgentBuild::COMPILER;
-
-        SerializedResponseNode compiler_flags;
-        compiler_flags.name = "flags";
-        compiler_flags.value = AgentBuild::COMPILER_FLAGS;
-
-        buildInfo.children.push_back(compiler_flags);
-        buildInfo.children.push_back(compiler_command);
-
-        buildInfo.children.push_back(build_version);
-        buildInfo.children.push_back(build_rev);
-        buildInfo.children.push_back(build_date);
-        desc.children.push_back(buildInfo);
         type.children.push_back(desc);
       }
       response.children.push_back(type);
@@ -474,12 +428,6 @@ class AgentManifest : public DeviceInformation {
       serialized.push_back(bundle);
     }
 
-    ComponentManifest compMan("default", nullptr);
-    // serialize the component information.
-    for (auto component : compMan.serialize()) {
-      serialized.push_back(component);
-    }
-
     return serialized;
   }
 };

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/e02304b3/libminifi/test/unit/PropertyValidationAgentInfoTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/PropertyValidationAgentInfoTests.cpp b/libminifi/test/unit/PropertyValidationAgentInfoTests.cpp
index 291f500..7a393c5 100644
--- a/libminifi/test/unit/PropertyValidationAgentInfoTests.cpp
+++ b/libminifi/test/unit/PropertyValidationAgentInfoTests.cpp
@@ -26,7 +26,7 @@
 #include "core/ClassLoader.h"
 
 TEST_CASE("Test Required", "[required]") {
-  minifi::state::response::ComponentManifest manifest("default");
+  minifi::state::response::ComponentManifest manifest("minifi-system");
   auto serialized = manifest.serialize();
   REQUIRE(serialized.size() > 0);
   const auto &resp = serialized[0];
@@ -45,7 +45,7 @@ TEST_CASE("Test Required", "[required]") {
 }
 
 TEST_CASE("Test Valid Regex", "[validRegex]") {
-  minifi::state::response::ComponentManifest manifest("default");
+  minifi::state::response::ComponentManifest manifest("minifi-system");
   auto serialized = manifest.serialize();
   REQUIRE(serialized.size() > 0);
   const auto &resp = serialized[0];
@@ -63,7 +63,7 @@ TEST_CASE("Test Valid Regex", "[validRegex]") {
 }
 
 TEST_CASE("Test Dependent", "[dependent]") {
-  minifi::state::response::ComponentManifest manifest("default");
+  minifi::state::response::ComponentManifest manifest("minifi-system");
   auto serialized = manifest.serialize();
   REQUIRE(serialized.size() > 0);
   const auto &resp = serialized[0];