You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/03/03 19:07:25 UTC

[GitHub] [nifi-minifi-cpp] msharee9 opened a new pull request #743: Minificpp 1169

msharee9 opened a new pull request #743: Minificpp 1169
URL: https://github.com/apache/nifi-minifi-cpp/pull/743
 
 
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r396394131
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 @szaszm I hacked together an ASAN build with clang-10 and it identified the issue most likely causing this:
   ```
   =================================================================
   ==58018==ERROR: AddressSanitizer: heap-use-after-free on address 0x603000000558 at pc 0x0000006a08df bp 0x7ffff1ddedf0 sp 0x7ffff1ddede8
   WRITE of size 4 at 0x603000000558 thread T54
       #0 0x6a08de in __gnu_cxx::__atomic_add(int volatile*, int) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:53:5
       #1 0x6a08de in __gnu_cxx::__atomic_add_dispatch(int*, int) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:96:7
       #2 0x6a08de in std::_Sp_counted_base<(__gnu_cxx::_Lock_policy)2>::_M_add_ref_copy() /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:139:9
       #3 0x6a08de in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count(std::__shared_count<(__gnu_cxx::_Lock_policy)2> const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:735:11
       #4 0x6a08de in std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2>::__shared_ptr(std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2> const&) /usr/bin/..
   /lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:1165:7
       #5 0x6a08de in std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator>::shared_ptr(std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator> const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/sha
   red_ptr.h:129:7
       #6 0x6a08de in org::apache::nifi::minifi::utils::IdGenerator::getIdGenerator() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/Id.h:156:12
       #7 0x69f0b6 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) /home/bakaid/nifi-minifi-cpp2/libminifi/include/core/Core.h:166:5
       #8 0x8667d8 in org::apache::nifi::minifi::core::Connectable::Connectable(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) /home/bakaid/nifi-minifi-cpp2/libminifi/src/core/Connectable.cpp:40:7
       #9 0xc6fb47 in org::apache::nifi::minifi::utils::HTTPClient::HTTPClient(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::shared_ptr<org::apache::nifi::minifi::controllers::SSLContextService>
   ) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/client/HTTPClient.cpp:36:7
       #10 0xc6758a in org::apache::nifi::minifi::c2::RESTSender::sendPayload(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, org::apache::nifi::minifi::c2::Direction, org::apache::nifi::minifi::c2::C2Payload
    const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RESTSender.cpp:106:21
       #11 0xc65637 in org::apache::nifi::minifi::c2::RESTSender::consumePayload(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::mini
   fi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RESTSender.cpp:69:10
       #12 0xc65c00 in org::apache::nifi::minifi::c2::RESTSender::consumePayload(org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::minifi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RE
   STSender.cpp
       #13 0xc65c00 in non-virtual thunk to org::apache::nifi::minifi::c2::RESTSender::consumePayload(org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::minifi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/h
   ttp-curl/protocols/RESTSender.cpp
       #14 0x7bb07d in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat() /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp:318:45
       #15 0x7d172a in org::apache::nifi::minifi::c2::C2Agent::C2Agent(std::shared_ptr<org::apache::nifi::minifi::core::controller::ControllerServiceProvider> const&, std::shared_ptr<org::apache::nifi::minifi::state::StateMonitor> const&, st
   d::shared_ptr<org::apache::nifi::minifi::Configure> const&, org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>&)::$_0::operator()() const /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Age
   nt.cpp:100:9
       #16 0x7d172a in std::_Function_handler<org::apache::nifi::minifi::utils::TaskRescheduleInfo (), org::apache::nifi::minifi::c2::C2Agent::C2Agent(std::shared_ptr<org::apache::nifi::minifi::core::controller::ControllerServiceProvider> co
   nst&, std::shared_ptr<org::apache::nifi::minifi::state::StateMonitor> const&, std::shared_ptr<org::apache::nifi::minifi::Configure> const&, org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>
   &)::$_0>::_M_invoke(std::_Any_data const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/std_function.h:282:9
       #17 0xa3652b in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/std_function.h:687:14
       #18 0xa3652b in org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/ThreadPool.h:97:16
       #19 0xa3652b in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run_tasks(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>) /home/bakaid/nifi-minifi-cpp2/libminifi/src
   /utils/ThreadPool.cpp:49:16
       #20 0xa64bfb in void std::__invoke_impl<void, void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache:
   :nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&>(std::__invoke_memfun_deref, void (org::apache::nifi::minifi::utils::ThreadPool<org
   ::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_pt
   r<org::apache::nifi::minifi::utils::WorkerThread>&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/invoke.h:73:14
       #21 0x7ffff76e86ee in execute_native_thread_routine /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../../src/libstdc++-v3/src/c++11/thread.cc:80:18
       #22 0x7ffff70756da in start_thread /build/glibc-OTsEL5/glibc-2.27/nptl/pthread_create.c:463
       #23 0x7ffff697e88e in clone /build/glibc-OTsEL5/glibc-2.27/misc/../sysdeps/unix/sysv/linux/x86_64/clone.S:95
   
   0x603000000558 is located 8 bytes inside of 24-byte region [0x603000000550,0x603000000568)
   freed by thread T0 here:
       #0 0x6861bd in operator delete(void*) (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x6861bd)
       #1 0x7ffff68a0040 in __run_exit_handlers /build/glibc-OTsEL5/glibc-2.27/stdlib/exit.c:108
   
   previously allocated by thread T0 here:
       #0 0x68595d in operator new(unsigned long) (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x68595d)
       #1 0x6a0bb4 in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count<org::apache::nifi::minifi::utils::IdGenerator*>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/
   c++/8/bits/shared_ptr_base.h:623:16
       #2 0x6a092a in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count<org::apache::nifi::minifi::utils::IdGenerator*>(org::apache::nifi::minifi::utils::IdGenerator*, std::integral_constant<bool, false>) /usr/bin/../lib/gcc/x8
   6_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:634:4
       #3 0x6a092a in std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2>::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, void>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/..
   /lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:1123:17
       #4 0x6a092a in std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator>::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, void>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../.
   ./../../include/c++/8/bits/shared_ptr.h:139:25
       #5 0x6a092a in org::apache::nifi::minifi::utils::IdGenerator::getIdGenerator() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/Id.h:155:53
       #6 0x552d2d in __cxx_global_var_init.10 /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp:44:62
       #7 0x552d2d in _GLOBAL__sub_I_C2Agent.cpp /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp
       #8 0x11a636c in __libc_csu_init (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x11a636c)
   
   Thread T54 created by T53 here:
       #0 0x640f9a in pthread_create (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x640f9a)
       #1 0x7ffff76e89a4 in __gthread_create /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/include/x86_64-linux-gnu/bits/gthr-default.h:662:35
       #2 0x7ffff76e89a4 in std::thread::_M_start_thread(std::unique_ptr<std::thread::_State, std::default_delete<std::thread::_State> >, void (*)()) /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../.
   ./src/libstdc++-v3/src/c++11/thread.cc:135:37
       #3 0xa358cc in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::createThread(std::function<void ()>&&) /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/ThreadPool.h:297:12
       #4 0xa30fb4 in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::manageWorkers() /home/bakaid/nifi-minifi-cpp2/libminifi/src/utils/ThreadPool.cpp:121:30
       #5 0x7ffff76e86ee in execute_native_thread_routine /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../../src/libstdc++-v3/src/c++11/thread.cc:80:18
   
   Thread T53 created by T0 here:
       #0 0x640f9a in pthread_create (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x640f9a)
       #1 0x7ffff76e89a4 in __gthread_create /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/include/x86_64-linux-gnu/bits/gthr-default.h:662:35
       #2 0x7ffff76e89a4 in std::thread::_M_start_thread(std::unique_ptr<std::thread::_State, std::default_delete<std::thread::_State> >, void (*)()) /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../.
   ./src/libstdc++-v3/src/c++11/thread.cc:135:37
       #3 0xaa0c75 in org::apache::nifi::minifi::FlowController::load(std::shared_ptr<org::apache::nifi::minifi::core::ProcessGroup> const&, bool) /home/bakaid/nifi-minifi-cpp2/libminifi/src/FlowController.cpp:320:20
       #4 0x68e993 in IntegrationBase::run(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >) /home/bakaid/nifi-minifi-cpp2/libminifi/test/integration/IntegrationBase.h:138:20
       #5 0x6948f3 in main /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/tests/C2JstackTest.cpp:76:11
       #6 0x7ffff687eb96 in __libc_start_main /build/glibc-OTsEL5/glibc-2.27/csu/../csu/libc-start.c:310
   
   SUMMARY: AddressSanitizer: heap-use-after-free /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:53:5 in __gnu_cxx::__atomic_add(int volatile*, int)
   Shadow bytes around the buggy address:
     0x0c067fff8050: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fd
     0x0c067fff8060: fa fa fd fd fd fd fa fa fd fd fd fd fa fa fd fd
     0x0c067fff8070: fd fd fa fa fd fd fd fd fa fa 00 00 00 fa fa fa
     0x0c067fff8080: fd fd fd fd fa fa fd fd fd fa fa fa fd fd fd fa
     0x0c067fff8090: fa fa fd fd fd fa fa fa fd fd fd fd fa fa fd fd
   =>0x0c067fff80a0: fd fd fa fa fd fd fd fa fa fa fd[fd]fd fa fa fa
     0x0c067fff80b0: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fd
     0x0c067fff80c0: fa fa fd fd fd fd fa fa fd fd fd fa fa fa fd fd
     0x0c067fff80d0: fd fa fa fa fd fd fd fa fa fa fd fd fd fa fa fa
     0x0c067fff80e0: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fa
     0x0c067fff80f0: fa fa fd fd fd fa fa fa fd fd fd fa fa fa fd fd
   Shadow byte legend (one shadow byte represents 8 application bytes):
     Addressable:           00
     Partially addressable: 01 02 03 04 05 06 07
     Heap left redzone:       fa
     Freed heap region:       fd
     Stack left redzone:      f1
     Stack mid redzone:       f2
     Stack right redzone:     f3
     Stack after return:      f5
     Stack use after scope:   f8
     Global redzone:          f9
     Global init order:       f6
     Poisoned by user:        f7
     Container overflow:      fc
     Array cookie:            ac
     Intra object redzone:    bb
     ASan internal:           fe
     Left alloca redzone:     ca
     Right alloca redzone:    cb
     Shadow gap:              cc
   ==58018==ABORTING
   [Thread 0x7fffefd6f700 (LWP 58080) exited]
   [Thread 0x7ffff059c700 (LWP 58079) exited]
   [Thread 0x7ffff0da5700 (LWP 58078) exited]
   [Thread 0x7ffff15bc700 (LWP 58077) exited]
   [Thread 0x7ffff1de1700 (LWP 58076) exited]
   [Thread 0x7ffff2e69700 (LWP 58074) exited]
   [Thread 0x7ffff7fd6fc0 (LWP 58018) exited]
   [Inferior 1 (process 58018) exited with code 01]
   ```
   
   What happens here is that the statically initialized variables get freed on `atexit` when the main thread stops. 
   Unfortunately, the threads executing the C2 functions seem to live longer than the main thread, and they try to use the IdGenerator and the Logger (both of which I have observed to crash), causing a use-after-free. This fits the symptoms perfectly, but the exact reason why the threadpool lives longer than the main thread has to be figured out (we have a lot of shared_ptr cycles, so destructors are not potentially called, we most likely have to manually ensure that these threads are stopped before exiting `main`).
   
   @msharee9 you will have to ensure that the C2 threads (and all threads) shut down before we exit main. Hopefully this is only an issue in IntegrationTests, but it might be an issue that is present in the real usage of the C2.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r396393474
 
 

 ##########
 File path: libminifi/include/c2/C2Agent.h
 ##########
 @@ -52,39 +55,28 @@ namespace c2 {
  *   0 HeartBeat --  RESERVED
  *   1-255 Defined by the configuration file.
  */
-class C2Agent : public state::UpdateController, public state::response::ResponseNodeSink, public std::enable_shared_from_this<C2Agent> {
+class C2Agent : public state::UpdateController {
  public:
 
-  C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<state::StateMonitor> &updateSink, const std::shared_ptr<Configure> &configure);
+  C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller,
+          const std::shared_ptr<state::StateMonitor> &updateSink,
+          const std::shared_ptr<Configure> &configure,
+          utils::ThreadPool<utils::TaskRescheduleInfo> &pool);
 
 Review comment:
   Using the `FlowController`'s thread pool for this can be dangerous.
   I do not want Processors to potentially starve C2. For example, if a processor gets stuck in a long onTrigger, I want to be able to reload the configuration (or at least restart the agent) through C2. This way Processors and the C2 has to share resources.
   I would prefer this to remain more like it was, with C2 owning its own threadpool, not sharing it (for the time being, with some refactor not even a threadpool would be needed).
   @arpadboda what's your take on this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393121094
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   The new C2JstackTest sometimes crashes (segfault) during UUID generation, in `IdGenerator::generateWithUuidImpl` because `uuid_impl_` is null for some reason. I don't think the root cause is in your changes but we need to address this before merging.
   The interesting part of the backtrace:
   ```
   #4  0x0000555555dc4bf3 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x5555566c5e70, mode=1, output=0x7ffff5720a40 "`\vr\365\377\177") at /home/szaszm/nifi-minifi-cpp-2/libminifi/src/utils/Id.cpp:288
   #5  0x0000555555dc4e6b in org::apache::nifi::minifi::utils::IdGenerator::generate (this=0x5555566c5e70, ident=...) at /home/szaszm/nifi-minifi-cpp-2/libminifi/src/utils/Id.cpp:332
   #6  0x0000555555bf5fc9 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent (this=0x7ffff5720de0, name="HTTPClient") at /home/szaszm/nifi-minifi-cpp-2/libminifi/include/core/Core.h:166
   #7  0x0000555555d02989 in org::apache::nifi::minifi::core::Connectable::Connectable (this=0x7ffff5720de0, name="HTTPClient") at /home/szaszm/nifi-minifi-cpp-2/libminifi/src/core/Connectable.cpp:43
   #8  0x0000555555ee28b2 in org::apache::nifi::minifi::utils::HTTPClient::HTTPClient (this=0x7ffff5720d70, url="https://localhost:41931/api/heartbeat", 
       ssl_context_service=std::shared_ptr<org::apache::nifi::minifi::controllers::SSLContextService> (empty) = {...}) at /home/szaszm/nifi-minifi-cpp-2/extensions/http-curl/client/HTTPClient.cpp:51
   #9  0x0000555555edc383 in org::apache::nifi::minifi::c2::RESTSender::sendPayload (this=0x5555568e5b40, url="https://localhost:41931/api/heartbeat", direction=org::apache::nifi::minifi::c2::TRANSMIT, payload=..., 
       outputConfig="{\n    \"operation\": \"heartbeat\",\n    \"agentInfo\": {\n        \"status\": {\n", ' ' <repeats 12 times>, "\"repositories\": {\n", ' ' <repeats 16 times>, "\"ff\": {\n", ' ' <repeats 20 times>, "\"size\": 0,\n", ' ' <repeats 20 times>, "\"running\": false,\n      "...) at /home/szaszm/nifi-minifi-cpp-2/extensions/http-curl/protocols/RESTSender.cpp:106
   #10 0x0000555555edbd61 in org::apache::nifi::minifi::c2::RESTSender::consumePayload (this=0x5555568e5b40, url="https://localhost:41931/api/heartbeat", payload=..., direction=org::apache::nifi::minifi::c2::TRANSMIT, async=false)
       at /home/szaszm/nifi-minifi-cpp-2/extensions/http-curl/protocols/RESTSender.cpp:69
   #11 0x0000555555edbeab in org::apache::nifi::minifi::c2::RESTSender::consumePayload (this=0x5555568e5b40, payload=..., direction=org::apache::nifi::minifi::c2::TRANSMIT, async=false)
       at /home/szaszm/nifi-minifi-cpp-2/extensions/http-curl/protocols/RESTSender.cpp:76
   #12 0x0000555555c90ad2 in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat (this=0x5555568db8b0) at /home/szaszm/nifi-minifi-cpp-2/libminifi/src/c2/C2Agent.cpp:318
   #13 0x0000555555c8dcbc in org::apache::nifi::minifi::c2::C2Agent::<lambda()>::operator()(void) const (__closure=0x7ffff5721b80) at /home/szaszm/nifi-minifi-cpp-2/libminifi/src/c2/C2Agent.cpp:100
   #14 0x0000555555c96478 in std::_Function_handler<org::apache::nifi::minifi::utils::TaskRescheduleInfo(), org::apache::nifi::minifi::c2::C2Agent::C2Agent(const std::shared_ptr<org::apache::nifi::minifi::core::controller::ControllerServiceProvider>&, const std::shared_ptr<org::apache::nifi::minifi::state::StateMonitor>&, const std::shared_ptr<org::apache::nifi::minifi::Configure>&, org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>&)::<lambda()> >::_M_invoke(const std::_Any_data &) (__functor=...) at /usr/include/c++/7/bits/std_function.h:302
   #15 0x0000555555ca6fc6 in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const (this=0x7ffff5721b80) at /usr/include/c++/7/bits/std_function.h:706
   #16 0x0000555555ca6c13 in org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run (this=0x7ffff5721b50) at /home/szaszm/nifi-minifi-cpp-2/libminifi/include/utils/ThreadPool.h:97
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395630595
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   This is very weird. You do not have to initialize the IdGenerator.
   That is, `utils::IdGenerator::getIdGenerator()` is a singleton factory function: the first time you call it, it creates a function static IdGenerator and returns it. This thread safe, protected by the function static initialization.
   
   After this, if you want (and in normal usage, you want), you can call `initialize` on the IdGenerator, supplying it properties, so that it can configure what method should be used for ID generation. This is not a thread safe function, and is called in both `MiNiFiMain.cpp` and in `TestBase` as one of the first things, before other threads are started.
   
   However, this is not a required step: if you don't call `initialize` it will just default to the time-based UUID implementation.
   
   `IdGenerator::uuid_impl_`, which on non-Windows platforms is a handle of the third party UUID implementation we use (OSSP UUID), is created in the `IdGenerator` constructor, so there should be no way for it to be nullptr (unless of course its constructor throws, or we get a bad_alloc, but we should have seen an exception in this case).
   
   `CoreComponent` uses the IdGenerator properly, in `libminifi/include/core/Core.h:166` it calls `utils::IdGenerator::getIdGenerator()` as it should, it does not store a static pointer, as far as I can see, @msharee9
   
   I could not reproduce this on macOS, so I am going to try to reproduce it on Linux.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389965534
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -406,22 +409,27 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
 
   std::chrono::steady_clock::time_point start_time_;
 
-  std::mutex metrics_mutex_;
+  mutable std::mutex metrics_mutex_;
 
 Review comment:
   👍 for this and making those functions const. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389963594
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPIntegrationBase.h
 ##########
 @@ -91,4 +87,64 @@ void CoapIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   }
 }
 
+class VerifyC2Base : public CoapIntegrationBase {
+ public:
+  explicit VerifyC2Base(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  virtual void testSetup() {
+    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setDebug<LogTestController>();
+  }
+
+  void runAssertions() {
+  }
+
+  virtual void queryRootProcessGroup(std::shared_ptr<core::ProcessGroup> pg) {
+    std::shared_ptr<core::Processor> proc = pg->findProcessor("invoke");
+    assert(proc != nullptr);
+
+    std::shared_ptr<minifi::processors::InvokeHTTP> inv = std::dynamic_pointer_cast<minifi::processors::InvokeHTTP>(proc);
 
 Review comment:
   I don't think this part and the above has any value here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395859336
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   This is a different but similar issue, @bakaid . I've verified that `uuid_impl_` was null by inserting an assert above the failing line, after seeing it in gdb with the first reproductions. My posted backtrace comes from code with this modification, that's why I skipped the top frames (as it's about assert and abort).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389961841
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
 
 Review comment:
   :suspect: 
   
   Why do we read size of blocksize in case the buffer size is 1024?
   My gut feeling says that this should be std::min or someting like that. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390614132
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -304,23 +305,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   virtual void enableAllControllerServices();
 
   /**
-   * Retrieves all root response nodes from this source.
-   * @param metric_vector -- metrics will be placed in this vector.
-   * @return result of the get operation.
-   *  0 Success
-   *  1 No error condition, but cannot obtain lock in timely manner.
-   *  -1 failure
+   * Retrieves metrics node
+   * @return metrics response node
    */
-  virtual int16_t getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass);
+  virtual std::shared_ptr<state::response::ResponseNode> getMetricsNode() const;
+
+  /**
+   * Retrieves root nodes configured to be included in heartbeat
+   * @param includeManifest -- determines if manifest is to be included
+   * @return a list of response nodes
+   */
+  virtual std::vector<std::shared_ptr<state::response::ResponseNode>> getHeartbeatNodes(bool includeManifest) const;
 
 Review comment:
   We can definitely have raw pointers in root_response_nodes_ but is there a reason not to use shared pointers?
   I understand we cannot use unique_ptrs here as we want to have the FlowController maintain ownership of the nodes and share it with C2Agent. But, given the choices of smart pointers we have shared_ptr is the best one we can use in this case. Having said that, I am not a strong adherent of using shared_ptr universally. In this case I think it is more of a coding guidelines/style/consistency in the code base.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393707126
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   That is interesting. Never saw the crash when I ran tests. However, I see Arpads point, it might be an issue of initialization order. I think instead of having a static pointer pointing to a singleton in class CoreComponent, we could just call out the static function that returns singleton. This way the singleton will be initialized whenever it is used.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r391746461
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -342,6 +342,31 @@ void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &nam
   }
 }
 
+/*
+void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container, bool is_collapsible) {
 
 Review comment:
   Committed that accidentally. Removed the comments in latest commit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r408217112
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -89,44 +93,68 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
         }
         request_mutex.unlock();
       }
-
-      if ( time_since > heart_beat_period_ ) {
-        last_run_ = now;
-        try {
-          performHeartBeat();
-        }
-        catch(const std::exception &e) {
-          logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
-        }
-        catch(...) {
-          logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
-        }
+      try {
+        performHeartBeat();
+      }
+      catch(const std::exception &e) {
+        logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
+      }
+      catch(...) {
+        logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
       }
 
       checkTriggers();
 
-      std::this_thread::sleep_for(std::chrono::milliseconds(heart_beat_period_ > 500 ? 500 : heart_beat_period_));
-      return state::Update(state::UpdateStatus(state::UpdateState::READ_COMPLETE, false));
+      return utils::TaskRescheduleInfo::RetryIn(std::chrono::milliseconds(heart_beat_period_));
     };
-
   functions_.push_back(c2_producer_);
 
   c2_consumer_ = [&]() {
     auto now = std::chrono::steady_clock::now();
     if ( queue_mutex.try_lock_until(now + std::chrono::seconds(1)) ) {
-      if (responses.size() > 0) {
-        const C2Payload payload(std::move(responses.back()));
-        responses.pop_back();
-        extractPayload(std::move(payload));
+      if (responses.empty()) {
+        queue_mutex.unlock();
+        return utils::TaskRescheduleInfo::RetryImmediately();
 
 Review comment:
   I guess in this case we constantly keep acquiring and releasing a lock in a threadpool worker, ending in a busy wait.
   Greta dislikes heating using cpu. :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r406886579
 
 

 ##########
 File path: libminifi/include/utils/BackTrace.h
 ##########
 @@ -81,13 +83,32 @@ void pull_trace(uint8_t frames_to_skip = 1);
  */
 void emplace_handler();
 
+class Lock {
 
 Review comment:
   std::unique_lock could be used here. Removed the custom Lock class. my bad.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r396391265
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
 
 Review comment:
   This is one of the bugs (and potential crashes) identified by ASAN.
   `buffer` is not guaranteed to contain a c-string, `mg_read` just reads binary data.
   The `std::string::append` overload that gets used here is the `string& append(const string& str, size_t pos, size_t count)` one, meaning that an `std::string` will be implicitly constructed from `buffer`, resulting in a stack overread and potential crash.
   It should be replaced with the `string& append( const CharT* s, size_type count )` overload since we are reading the string from `pos == 0` anyway.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r409396297
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -89,44 +93,68 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
         }
         request_mutex.unlock();
       }
-
-      if ( time_since > heart_beat_period_ ) {
-        last_run_ = now;
-        try {
-          performHeartBeat();
-        }
-        catch(const std::exception &e) {
-          logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
-        }
-        catch(...) {
-          logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
-        }
+      try {
+        performHeartBeat();
+      }
+      catch(const std::exception &e) {
+        logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
+      }
+      catch(...) {
+        logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
       }
 
       checkTriggers();
 
-      std::this_thread::sleep_for(std::chrono::milliseconds(heart_beat_period_ > 500 ? 500 : heart_beat_period_));
-      return state::Update(state::UpdateStatus(state::UpdateState::READ_COMPLETE, false));
+      return utils::TaskRescheduleInfo::RetryIn(std::chrono::milliseconds(heart_beat_period_));
     };
-
   functions_.push_back(c2_producer_);
 
   c2_consumer_ = [&]() {
     auto now = std::chrono::steady_clock::now();
     if ( queue_mutex.try_lock_until(now + std::chrono::seconds(1)) ) {
-      if (responses.size() > 0) {
-        const C2Payload payload(std::move(responses.back()));
-        responses.pop_back();
-        extractPayload(std::move(payload));
+      if (responses.empty()) {
+        queue_mutex.unlock();
+        return utils::TaskRescheduleInfo::RetryImmediately();
 
 Review comment:
   RetryImmediately was introduced for a reason: when something takes quite long (an ontrigger call of a processor for eg), it might happen that the next schedule is already in the past. Although we cannot continue executing it as that would result in starvation. 
   In this case it's fair to reschedule immediately. 
   This means that the given task is put back to the end of the queue, so in case there are tasks already waiting for CPU, those are executed first, otherwise this one can be continued immediately without unnecessary wait. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390463141
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPIntegrationBase.h
 ##########
 @@ -91,4 +87,64 @@ void CoapIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   }
 }
 
+class VerifyC2Base : public CoapIntegrationBase {
+ public:
+  explicit VerifyC2Base(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  virtual void testSetup() {
+    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setDebug<LogTestController>();
+  }
+
+  void runAssertions() {
 
 Review comment:
   It's already declared to be pure virtual by the base class. I'd just omit it from this class definition.
   
   Supporting guideline: "33. Make non-leaf classes abstract" from More Effective C++ (1996), Scott Meyers

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r408289302
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -89,44 +93,68 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
         }
         request_mutex.unlock();
       }
-
-      if ( time_since > heart_beat_period_ ) {
-        last_run_ = now;
-        try {
-          performHeartBeat();
-        }
-        catch(const std::exception &e) {
-          logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
-        }
-        catch(...) {
-          logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
-        }
+      try {
+        performHeartBeat();
+      }
+      catch(const std::exception &e) {
+        logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
+      }
+      catch(...) {
+        logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
       }
 
       checkTriggers();
 
-      std::this_thread::sleep_for(std::chrono::milliseconds(heart_beat_period_ > 500 ? 500 : heart_beat_period_));
-      return state::Update(state::UpdateStatus(state::UpdateState::READ_COMPLETE, false));
+      return utils::TaskRescheduleInfo::RetryIn(std::chrono::milliseconds(heart_beat_period_));
     };
-
   functions_.push_back(c2_producer_);
 
   c2_consumer_ = [&]() {
     auto now = std::chrono::steady_clock::now();
     if ( queue_mutex.try_lock_until(now + std::chrono::seconds(1)) ) {
-      if (responses.size() > 0) {
-        const C2Payload payload(std::move(responses.back()));
-        responses.pop_back();
-        extractPayload(std::move(payload));
+      if (responses.empty()) {
+        queue_mutex.unlock();
+        return utils::TaskRescheduleInfo::RetryImmediately();
 
 Review comment:
   
   
   
   
   > @msharee9 Tried lightweight and normal heartbeat and different flows, they seem to work fine.
   > 
   > However, even when running with no flow, but C2 enabled, minifi consumes 150% CPU both on macOS and Linux.
   > 
   > This is very likely caused by the issue pointed out by Arpad, that is, we are busy looping because of an incorrect usage of the Thread Pool.
   > 
   > This callgrind profiling of minifi also seems to reinforce that suspicion:
   > <img alt="Screenshot 2020-04-14 at 17 21 42" width="1679" src="https://user-images.githubusercontent.com/9108564/79242487-b1a66680-7e74-11ea-9a6b-f07f61bccc54.png">
   
   Changed it to Retry in 100 ms. I guess then the RetryImmediately() cannot be used anywhere. We should either remove it altogether or have some delay before we call continue at this line.
   https://github.com/apache/nifi-minifi-cpp/blob/3116eb817a7182486bf3e50b87c3032ac2074031/libminifi/src/utils/ThreadPool.cpp#L53

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395811680
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   The above happened on Ubuntu 18.04 with GCC 7.5 (system-level, from `build-essential`). It hangs when being run through `ctest` and segfaults when run with `gdb`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392204665
 
 

 ##########
 File path: extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
 ##########
 @@ -195,12 +130,23 @@ int main(int argc, char **argv) {
   if (url.find("https") != std::string::npos) {
     isSecure = true;
   }
+  {
+    VerifyC2Heartbeat harness(isSecure);
+
+    harness.setKeyDir(key_dir);
+
+    HeartbeatHandler responder(isSecure);
+
+    harness.setUrl(url, &responder);
+
+    harness.run(test_file_location);
+  }
 
 Review comment:
   I think these statements logically fit together, so I see no reason for separating them with empty lines.
   
   The google style guide (which we reference in our CONTRIB.md) is even more conservative than I am: https://google.github.io/styleguide/cppguide.html#Vertical_Whitespace

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395865738
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   Thank you, sounds fair, I have occasionally seen it manifest in other ways, so it is reasonable to assume that it occurs in different ways for us based on our environment. And it reinforces my memory corruption hunch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389004654
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -449,13 +451,9 @@ void FlowController::initializeC2() {
 
   if (!c2_initialized_) {
     configuration_->setAgentIdentifier(identifier_str);
-    state::StateManager::initialize();
-    std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+    c2_agent_ = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
 
 Review comment:
   As discussed, we will take this in another PR as it requires some major changes that might break interfaces.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389965084
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -304,23 +305,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   virtual void enableAllControllerServices();
 
   /**
-   * Retrieves all root response nodes from this source.
-   * @param metric_vector -- metrics will be placed in this vector.
-   * @return result of the get operation.
-   *  0 Success
-   *  1 No error condition, but cannot obtain lock in timely manner.
-   *  -1 failure
+   * Retrieves metrics node
+   * @return metrics response node
    */
-  virtual int16_t getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass);
+  virtual std::shared_ptr<state::response::ResponseNode> getMetricsNode() const;
+
+  /**
+   * Retrieves root nodes configured to be included in heartbeat
+   * @param includeManifest -- determines if manifest is to be included
+   * @return a list of response nodes
+   */
+  virtual std::vector<std::shared_ptr<state::response::ResponseNode>> getHeartbeatNodes(bool includeManifest) const;
 
 Review comment:
   Do these really need to be shared?
   Do we ever copy the metrics, heartbeat nodes and the manifest?
   
   Please take it as a question, so in case the answer is yes, I'm totally fine with this.
   If we don't copy, just generate and send, I would prefer using unique ptrs to have a clear indication of ownership.  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on issue #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on issue #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#issuecomment-601870193
 
 
   > `C2FailedUpdateTest` fails for me consistently on macOS, and also seems to fail on Travis.
   > Also, the C2JstackTest never seems to return on Travis, that's why jobs are timing out (I could not reproduce this locally).
   > I am going to review the review fixes next, but these test failures must be addressed.
   
   It fails because we have an unnecessary and out of context assert to test the number of calls which somehow depends on the number of times the client POSTs to the CivetServer.
   assert(h_ex.calls_ <= (some time duration) + 1);
   I think that line came as a copy/paste from some other tests. It is certainly not needed here and also does not test anything related to the C2 failed update itself.
   
   The number of calls to handlePost should not be tested against some arbitrary amount of time duration. It does not make sense at all.
   
   Also I created a follow up ticket to cleanup the C2 Update tests. There is a lot of code duplication between some C2 update tests.
   
   https://issues.apache.org/jira/browse/MINIFICPP-1183

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390477569
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -304,23 +305,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   virtual void enableAllControllerServices();
 
   /**
-   * Retrieves all root response nodes from this source.
-   * @param metric_vector -- metrics will be placed in this vector.
-   * @return result of the get operation.
-   *  0 Success
-   *  1 No error condition, but cannot obtain lock in timely manner.
-   *  -1 failure
+   * Retrieves metrics node
+   * @return metrics response node
    */
-  virtual int16_t getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass);
+  virtual std::shared_ptr<state::response::ResponseNode> getMetricsNode() const;
+
+  /**
+   * Retrieves root nodes configured to be included in heartbeat
+   * @param includeManifest -- determines if manifest is to be included
+   * @return a list of response nodes
+   */
+  virtual std::vector<std::shared_ptr<state::response::ResponseNode>> getHeartbeatNodes(bool includeManifest) const;
 
 Review comment:
   (response to @arpadboda)
   I think `root_response_nodes_` owns the objects and this function shares this ownership with the callers. Whether this shared ownership is needed or not is not clear to me.
   We can definitely not use `unique_ptr`s for the returned pointers here, as ownership belongs to `root_response_nodes_`, but we could maybe use `state::response::ResponseNode*` (i.e. observer ptr) if the lifetime of the nodes don't necessitate shared ownership.
   
   Additionally, `root_response_nodes_` is protected, which means it's part of our API and contains `shared_ptr`s, so we're not going to get `unique_ptr`s there.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r396394131
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 @szaszm I hacked together an ASAN build with clang-10 and it identified the issue most likely causing this:
   ```
   =================================================================
   ==58018==ERROR: AddressSanitizer: heap-use-after-free on address 0x603000000558 at pc 0x0000006a08df bp 0x7ffff1ddedf0 sp 0x7ffff1ddede8
   WRITE of size 4 at 0x603000000558 thread T54
       #0 0x6a08de in __gnu_cxx::__atomic_add(int volatile*, int) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:53:5
       #1 0x6a08de in __gnu_cxx::__atomic_add_dispatch(int*, int) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:96:7
       #2 0x6a08de in std::_Sp_counted_base<(__gnu_cxx::_Lock_policy)2>::_M_add_ref_copy() /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:139:9
       #3 0x6a08de in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count(std::__shared_count<(__gnu_cxx::_Lock_policy)2> const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:735:11
       #4 0x6a08de in std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2>::__shared_ptr(std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2> const&) /usr/bin/..
   /lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:1165:7
       #5 0x6a08de in std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator>::shared_ptr(std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator> const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/sha
   red_ptr.h:129:7
       #6 0x6a08de in org::apache::nifi::minifi::utils::IdGenerator::getIdGenerator() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/Id.h:156:12
       #7 0x69f0b6 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) /home/bakaid/nifi-minifi-cpp2/libminifi/include/core/Core.h:166:5
       #8 0x8667d8 in org::apache::nifi::minifi::core::Connectable::Connectable(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) /home/bakaid/nifi-minifi-cpp2/libminifi/src/core/Connectable.cpp:40:7
       #9 0xc6fb47 in org::apache::nifi::minifi::utils::HTTPClient::HTTPClient(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, std::shared_ptr<org::apache::nifi::minifi::controllers::SSLContextService>
   ) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/client/HTTPClient.cpp:36:7
       #10 0xc6758a in org::apache::nifi::minifi::c2::RESTSender::sendPayload(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >, org::apache::nifi::minifi::c2::Direction, org::apache::nifi::minifi::c2::C2Payload
    const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RESTSender.cpp:106:21
       #11 0xc65637 in org::apache::nifi::minifi::c2::RESTSender::consumePayload(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::mini
   fi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RESTSender.cpp:69:10
       #12 0xc65c00 in org::apache::nifi::minifi::c2::RESTSender::consumePayload(org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::minifi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/protocols/RE
   STSender.cpp
       #13 0xc65c00 in non-virtual thunk to org::apache::nifi::minifi::c2::RESTSender::consumePayload(org::apache::nifi::minifi::c2::C2Payload const&, org::apache::nifi::minifi::c2::Direction, bool) /home/bakaid/nifi-minifi-cpp2/extensions/h
   ttp-curl/protocols/RESTSender.cpp
       #14 0x7bb07d in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat() /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp:318:45
       #15 0x7d172a in org::apache::nifi::minifi::c2::C2Agent::C2Agent(std::shared_ptr<org::apache::nifi::minifi::core::controller::ControllerServiceProvider> const&, std::shared_ptr<org::apache::nifi::minifi::state::StateMonitor> const&, st
   d::shared_ptr<org::apache::nifi::minifi::Configure> const&, org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>&)::$_0::operator()() const /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Age
   nt.cpp:100:9
       #16 0x7d172a in std::_Function_handler<org::apache::nifi::minifi::utils::TaskRescheduleInfo (), org::apache::nifi::minifi::c2::C2Agent::C2Agent(std::shared_ptr<org::apache::nifi::minifi::core::controller::ControllerServiceProvider> co
   nst&, std::shared_ptr<org::apache::nifi::minifi::state::StateMonitor> const&, std::shared_ptr<org::apache::nifi::minifi::Configure> const&, org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>
   &)::$_0>::_M_invoke(std::_Any_data const&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/std_function.h:282:9
       #17 0xa3652b in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/std_function.h:687:14
       #18 0xa3652b in org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/ThreadPool.h:97:16
       #19 0xa3652b in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run_tasks(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>) /home/bakaid/nifi-minifi-cpp2/libminifi/src
   /utils/ThreadPool.cpp:49:16
       #20 0xa64bfb in void std::__invoke_impl<void, void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache:
   :nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&>(std::__invoke_memfun_deref, void (org::apache::nifi::minifi::utils::ThreadPool<org
   ::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_pt
   r<org::apache::nifi::minifi::utils::WorkerThread>&) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/invoke.h:73:14
       #21 0x7ffff76e86ee in execute_native_thread_routine /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../../src/libstdc++-v3/src/c++11/thread.cc:80:18
       #22 0x7ffff70756da in start_thread /build/glibc-OTsEL5/glibc-2.27/nptl/pthread_create.c:463
       #23 0x7ffff697e88e in clone /build/glibc-OTsEL5/glibc-2.27/misc/../sysdeps/unix/sysv/linux/x86_64/clone.S:95
   
   0x603000000558 is located 8 bytes inside of 24-byte region [0x603000000550,0x603000000568)
   freed by thread T0 here:
       #0 0x6861bd in operator delete(void*) (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x6861bd)
       #1 0x7ffff68a0040 in __run_exit_handlers /build/glibc-OTsEL5/glibc-2.27/stdlib/exit.c:108
   
   previously allocated by thread T0 here:
       #0 0x68595d in operator new(unsigned long) (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x68595d)
       #1 0x6a0bb4 in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count<org::apache::nifi::minifi::utils::IdGenerator*>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/
   c++/8/bits/shared_ptr_base.h:623:16
       #2 0x6a092a in std::__shared_count<(__gnu_cxx::_Lock_policy)2>::__shared_count<org::apache::nifi::minifi::utils::IdGenerator*>(org::apache::nifi::minifi::utils::IdGenerator*, std::integral_constant<bool, false>) /usr/bin/../lib/gcc/x8
   6_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:634:4
       #3 0x6a092a in std::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, (__gnu_cxx::_Lock_policy)2>::__shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, void>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/..
   /lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/bits/shared_ptr_base.h:1123:17
       #4 0x6a092a in std::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator>::shared_ptr<org::apache::nifi::minifi::utils::IdGenerator, void>(org::apache::nifi::minifi::utils::IdGenerator*) /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../.
   ./../../include/c++/8/bits/shared_ptr.h:139:25
       #5 0x6a092a in org::apache::nifi::minifi::utils::IdGenerator::getIdGenerator() /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/Id.h:155:53
       #6 0x552d2d in __cxx_global_var_init.10 /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp:44:62
       #7 0x552d2d in _GLOBAL__sub_I_C2Agent.cpp /home/bakaid/nifi-minifi-cpp2/libminifi/src/c2/C2Agent.cpp
       #8 0x11a636c in __libc_csu_init (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x11a636c)
   
   Thread T54 created by T53 here:
       #0 0x640f9a in pthread_create (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x640f9a)
       #1 0x7ffff76e89a4 in __gthread_create /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/include/x86_64-linux-gnu/bits/gthr-default.h:662:35
       #2 0x7ffff76e89a4 in std::thread::_M_start_thread(std::unique_ptr<std::thread::_State, std::default_delete<std::thread::_State> >, void (*)()) /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../.
   ./src/libstdc++-v3/src/c++11/thread.cc:135:37
       #3 0xa358cc in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::createThread(std::function<void ()>&&) /home/bakaid/nifi-minifi-cpp2/libminifi/include/utils/ThreadPool.h:297:12
       #4 0xa30fb4 in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::manageWorkers() /home/bakaid/nifi-minifi-cpp2/libminifi/src/utils/ThreadPool.cpp:121:30
       #5 0x7ffff76e86ee in execute_native_thread_routine /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../../src/libstdc++-v3/src/c++11/thread.cc:80:18
   
   Thread T53 created by T0 here:
       #0 0x640f9a in pthread_create (/home/bakaid/nifi-minifi-cpp2/build/extensions/http-curl/tests/C2JstackTest+0x640f9a)
       #1 0x7ffff76e89a4 in __gthread_create /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/include/x86_64-linux-gnu/bits/gthr-default.h:662:35
       #2 0x7ffff76e89a4 in std::thread::_M_start_thread(std::unique_ptr<std::thread::_State, std::default_delete<std::thread::_State> >, void (*)()) /build/gcc-8-uSa_z7/gcc-8-8.3.0/build/x86_64-linux-gnu/libstdc++-v3/src/c++11/../../../../.
   ./src/libstdc++-v3/src/c++11/thread.cc:135:37
       #3 0xaa0c75 in org::apache::nifi::minifi::FlowController::load(std::shared_ptr<org::apache::nifi::minifi::core::ProcessGroup> const&, bool) /home/bakaid/nifi-minifi-cpp2/libminifi/src/FlowController.cpp:320:20
       #4 0x68e993 in IntegrationBase::run(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >) /home/bakaid/nifi-minifi-cpp2/libminifi/test/integration/IntegrationBase.h:138:20
       #5 0x6948f3 in main /home/bakaid/nifi-minifi-cpp2/extensions/http-curl/tests/C2JstackTest.cpp:76:11
       #6 0x7ffff687eb96 in __libc_start_main /build/glibc-OTsEL5/glibc-2.27/csu/../csu/libc-start.c:310
   
   SUMMARY: AddressSanitizer: heap-use-after-free /usr/bin/../lib/gcc/x86_64-linux-gnu/8/../../../../include/c++/8/ext/atomicity.h:53:5 in __gnu_cxx::__atomic_add(int volatile*, int)
   Shadow bytes around the buggy address:
     0x0c067fff8050: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fd
     0x0c067fff8060: fa fa fd fd fd fd fa fa fd fd fd fd fa fa fd fd
     0x0c067fff8070: fd fd fa fa fd fd fd fd fa fa 00 00 00 fa fa fa
     0x0c067fff8080: fd fd fd fd fa fa fd fd fd fa fa fa fd fd fd fa
     0x0c067fff8090: fa fa fd fd fd fa fa fa fd fd fd fd fa fa fd fd
   =>0x0c067fff80a0: fd fd fa fa fd fd fd fa fa fa fd[fd]fd fa fa fa
     0x0c067fff80b0: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fd
     0x0c067fff80c0: fa fa fd fd fd fd fa fa fd fd fd fa fa fa fd fd
     0x0c067fff80d0: fd fa fa fa fd fd fd fa fa fa fd fd fd fa fa fa
     0x0c067fff80e0: fd fd fd fa fa fa fd fd fd fa fa fa fd fd fd fa
     0x0c067fff80f0: fa fa fd fd fd fa fa fa fd fd fd fa fa fa fd fd
   Shadow byte legend (one shadow byte represents 8 application bytes):
     Addressable:           00
     Partially addressable: 01 02 03 04 05 06 07
     Heap left redzone:       fa
     Freed heap region:       fd
     Stack left redzone:      f1
     Stack mid redzone:       f2
     Stack right redzone:     f3
     Stack after return:      f5
     Stack use after scope:   f8
     Global redzone:          f9
     Global init order:       f6
     Poisoned by user:        f7
     Container overflow:      fc
     Array cookie:            ac
     Intra object redzone:    bb
     ASan internal:           fe
     Left alloca redzone:     ca
     Right alloca redzone:    cb
     Shadow gap:              cc
   ==58018==ABORTING
   [Thread 0x7fffefd6f700 (LWP 58080) exited]
   [Thread 0x7ffff059c700 (LWP 58079) exited]
   [Thread 0x7ffff0da5700 (LWP 58078) exited]
   [Thread 0x7ffff15bc700 (LWP 58077) exited]
   [Thread 0x7ffff1de1700 (LWP 58076) exited]
   [Thread 0x7ffff2e69700 (LWP 58074) exited]
   [Thread 0x7ffff7fd6fc0 (LWP 58018) exited]
   [Inferior 1 (process 58018) exited with code 01]
   ```
   
   What happens here is that the statically initialized variables get freed on `atexit` when the main thread stops. 
   Unfortunately, the threads executing the C2 functions seem to live longer than the main thread, and they try to use the IdGenerator and the Logger (both of which I have observed to crash), causing a use-after-free. This fits the symptoms perfectly, but the exact reason why the threadpool lives longer than the main thread has to be figured out (we have a lot of shared_ptr cycles, so destructors are not potentially called, we most likely have to manually ensure that these threads are stopped before exiting `main`.
   
   @msharee9 you will have to ensure that the C2 threads (and all threads) shut down before we exit main. Hopefully this is only an issue in IntegrationTests, but it might be an issue that is present in the real usage of the C2.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r388283597
 
 

 ##########
 File path: libminifi/include/utils/ThreadPool.h
 ##########
 @@ -412,6 +414,22 @@ class ThreadPool {
   void run_tasks(std::shared_ptr<WorkerThread> thread);
 };
 
+template<typename T>
+bool ThreadPool<T>::execute(Worker<T> &&task) {
 
 Review comment:
   If this is going to remain in the final version, the non-future returning and future returning version should be deduplicated.
   Also, @arpadboda is currently working on a ThreadPool refactor/fix, I think it would be prudent to discuss any change to ThreadPool (and any new usage) with him.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390614132
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -304,23 +305,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   virtual void enableAllControllerServices();
 
   /**
-   * Retrieves all root response nodes from this source.
-   * @param metric_vector -- metrics will be placed in this vector.
-   * @return result of the get operation.
-   *  0 Success
-   *  1 No error condition, but cannot obtain lock in timely manner.
-   *  -1 failure
+   * Retrieves metrics node
+   * @return metrics response node
    */
-  virtual int16_t getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass);
+  virtual std::shared_ptr<state::response::ResponseNode> getMetricsNode() const;
+
+  /**
+   * Retrieves root nodes configured to be included in heartbeat
+   * @param includeManifest -- determines if manifest is to be included
+   * @return a list of response nodes
+   */
+  virtual std::vector<std::shared_ptr<state::response::ResponseNode>> getHeartbeatNodes(bool includeManifest) const;
 
 Review comment:
   We can definitely have raw pointers in root_response_nodes_ but is there a reason not to use shared pointers?
   I understand we cannot use unique_ptrs here as we want to have the FlowController maintain ownership of the nodes. But, given the choices of smart pointers we have shared_ptr is the best one we can use in this case. Having said that, I am not a strong adherent of using shared_ptr universally. In this case I think it is more of a coding guidelines/style/consistency in the code base.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392347808
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -907,29 +908,44 @@ int16_t FlowController::clearConnection(const std::string &connection) {
   return -1;
 }
 
-int16_t FlowController::getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
+std::shared_ptr<state::response::ResponseNode> FlowController::getMetricsNode(const std::string& metricsClass) const {
   std::lock_guard<std::mutex> lock(metrics_mutex_);
-
-  for (auto metric : root_response_nodes_) {
-    metric_vector.push_back(metric.second);
+  if (!metricsClass.empty()) {
+    const auto citer = component_metrics_.find(metricsClass);
+    if (citer != component_metrics_.end()) {
+      return citer->second;
+    }
+  } else {
+    const auto iter = root_response_nodes_.find("metrics");
+    if (iter != root_response_nodes_.end()) {
+      return iter->second;
+    }
 
 Review comment:
   There is no code duplication as far as I see. we are looking up different maps in if and else branches.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390456234
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"FlowController\"  } ]}";
 
 Review comment:
   Using a raw string literal would be nice for hardcoded JSONs.
   We can use the preprocessor for concatenation of string literals.
   We can use char arrays to avoid heap allocations. I wish we had `std::string_view`.
   I'm not sure about wrapping and copy-initialization vs direct-list-initialization.
   ```cpp
   const char heartbeat_response[] = R"json({"operation": "heartbeat", "requested_operations: [)json"
       R"json({"operationid": 41, "operation": "stop", "operand": "invoke"}, )json"
       R"json({"operationid": 42, "operation": "stop", "operand": "FlowController"}]})json";
   const char heartbeat_response2[]{ R"json({"operation": "heartbeat", "requested_operations: [{"operationid": 41, "operation": "stop", "operand": "invoke"}, {"operationid": 42, "operation": "stop", "operand": "FlowController"}]})json" };
   const auto response_length = sizeof(heartbeat_response);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r391745333
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -173,9 +173,9 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
 
 FlowController::~FlowController() {
   stop(true);
+  c2_agent_->stop();
 
 Review comment:
   No we cannot avoid null check here. Fixed it in latest commit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395838661
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @bakaid The CoreComponent before this change had a static pointer to utils::IdGenerator.
   And thats where the problem was. The call to generate(uuid_) could well happen before the idgenerator_ was initialized.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392364508
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == true);
+    assert(root["agentInfo"]["agentManifest"].HasMember("bundles") == true);
 
 Review comment:
   redundant `== true`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393054891
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
 
 Review comment:
   unused parameter 'root'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390615024
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -406,22 +409,27 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
 
   std::chrono::steady_clock::time_point start_time_;
 
-  std::mutex metrics_mutex_;
+  mutable std::mutex metrics_mutex_;
   // root_nodes cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
+
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> device_information_;
 
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> component_metrics_;
 
   std::map<uint8_t, std::vector<std::shared_ptr<state::response::ResponseNode>>> component_metrics_by_id_;
+
   // metrics last run
   std::chrono::steady_clock::time_point last_metrics_capture_;
 
  private:
   std::shared_ptr<logging::Logger> logger_;
   std::string serial_number_;
+
+  std::shared_ptr<state::UpdateController> c2_agent_;
 
 Review comment:
   You have a valid point. Made it a unique_ptr

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r391742620
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -342,6 +342,31 @@ void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &nam
   }
 }
 
+/*
+void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container, bool is_collapsible) {
 
 Review comment:
   What about this, why comment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392275028
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -601,6 +593,15 @@ void FlowController::initializeC2() {
   }
 
   loadC2ResponseConfiguration();
+
+  if (!c2_initialized_) {
+    c2_agent_ = std::unique_ptr<c2::C2Agent>(new c2::C2Agent(std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+                                                             std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+                                                             configuration_,
+                                                             thread_pool_));
 
 Review comment:
   I've added a `make_unique` implementation in `GeneralUtils.h`. Use it if you're interested.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392208226
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
 
 Review comment:
   ES.87: Don’t add redundant == or != to conditions
   http://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#es87-dont-add-redundant--or--to-conditions
   
   I personally don't mind comparison against `nullptr`, but trying to change my habits to avoid it. On the other hand comparison of boolean values against literal `true` or `false` just increases clutter without any benefit IMO.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392359958
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -172,6 +173,9 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
 
 FlowController::~FlowController() {
   stop(true);
+  if (c2_agent_)
+    c2_agent_->stop();
+  thread_pool_.shutdown();
 
 Review comment:
   Ah, the github folding confused me. Thanks for the clarification.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392346281
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -172,6 +173,9 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
 
 FlowController::~FlowController() {
   stop(true);
+  if (c2_agent_)
+    c2_agent_->stop();
+  thread_pool_.shutdown();
 
 Review comment:
   We do stop the schedulers before stopping threadpool here. Look at line # 175.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392257098
 
 

 ##########
 File path: libminifi/include/core/state/nodes/AgentInformation.h
 ##########
 @@ -68,7 +68,7 @@ namespace response {
 
 class ComponentManifest : public DeviceInformation {
  public:
-  ComponentManifest(std::string name, utils::Identifier & uuid)
+  ComponentManifest(const std::string& name, utils::Identifier & uuid)
 
 Review comment:
   :+1: 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r406770245
 
 

 ##########
 File path: libminifi/src/utils/BackTrace.cpp
 ##########
 @@ -80,25 +80,21 @@ BackTrace TraceResolver::getBackTrace(std::string thread_name, std::thread::nati
   // lock so that we only perform one backtrace at a time.
 #ifdef HAS_EXECINFO
   std::lock_guard<std::mutex> lock(mutex_);
-
-  caller_handle_ = pthread_self();
-  thread_handle_ = thread_handle;
   trace_ = BackTrace(std::move(thread_name));
 
-  if (0 == thread_handle_ || pthread_equal(caller_handle_, thread_handle)) {
+  if (0 == thread_handle || pthread_equal(pthread_self(), thread_handle)) {
     pull_trace();
   } else {
-    if (thread_handle_ == 0) {
+    if (thread_handle == 0) {
       return std::move(trace_);
     }
     emplace_handler();
-    if (pthread_kill(thread_handle_, SIGUSR2) != 0) {
+    std::unique_lock<std::mutex> ulock(trace_mutex_);
+    if (pthread_kill(thread_handle, SIGUSR2) != 0) {
       return std::move(trace_);
     }
-    sigset_t mask;
-    sigfillset(&mask);
-    sigdelset(&mask, SIGUSR2);
-    sigsuspend(&mask);
 
 Review comment:
   Nice catch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395630595
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   This is very weird. You do not have to initialize the IdGenerator.
   That is, `utils::IdGenerator::getIdGenerator()` is a singleton factory function: the first time you call it, it creates a function static IdGenerator and returns it. This thread safe, protected by the function static initialization.
   
   After this, if you want (and in normal usage, you want), you can call `initialize` on the IdGenerator, supplying it properties, so that it can configure what method should be used for ID generation. This is not a thread safe function, and is called in both `MiNiFiMain.cpp` and in `TestBase` as one of the first things, before other threads are started.
   
   However, this is not a required step: if you don't call `initialize` it will just default to the time-based UUID implementation.
   
   `uuid_impl_`, which on non-Windows platforms is a handle of the third party UUID implementation we use (OSSP UUID), is created in the `IdGenerator` constructor, so there should be no way for it to be nullptr (unless of course its constructor throws, or we get a bad_alloc, but we should have seen an exception in this case).
   
   `CoreComponent` uses the IdGenerator properly, in `libminifi/include/core/Core.h:166` it calls `utils::IdGenerator::getIdGenerator()` as it should, it does not store a static pointer, as far as I can see, @msharee9
   
   I could not reproduce this on macOS, so I am going to try to reproduce it on Linux.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392277388
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -907,29 +908,44 @@ int16_t FlowController::clearConnection(const std::string &connection) {
   return -1;
 }
 
-int16_t FlowController::getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
+std::shared_ptr<state::response::ResponseNode> FlowController::getMetricsNode(const std::string& metricsClass) const {
   std::lock_guard<std::mutex> lock(metrics_mutex_);
-
-  for (auto metric : root_response_nodes_) {
-    metric_vector.push_back(metric.second);
+  if (!metricsClass.empty()) {
+    const auto citer = component_metrics_.find(metricsClass);
+    if (citer != component_metrics_.end()) {
+      return citer->second;
+    }
+  } else {
+    const auto iter = root_response_nodes_.find("metrics");
+    if (iter != root_response_nodes_.end()) {
+      return iter->second;
+    }
 
 Review comment:
   It's possible to avoid code duplication by saying:
   `const char* const metrics_key = !metricsClass.empty() ? metricsClass.c_str() : "metrics";`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389013212
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -449,13 +451,9 @@ void FlowController::initializeC2() {
 
   if (!c2_initialized_) {
     configuration_->setAgentIdentifier(identifier_str);
-    state::StateManager::initialize();
-    std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+    c2_agent_ = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
 
 Review comment:
   The base class changes of `FlowController` also break API backwards compatibility. If we care about bw compat in this PR, we should revert that as well. If we don't, then we should probably go ahead with the above as well.
   
   I was not part of the cited discussion, so I might be missing something.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393057505
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *) {
+    verifyJsonHasAgentManifest(root);
+  }
+
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+  }
+
+  void verify(struct mg_connection *conn) {
+    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());
+      std::string operation = root["operation"].GetString();
+      if (operation == "heartbeat") {
+        handleHeartbeat(root, conn);
+      } else if (operation == "acknowledge") {
+        handleAcknowledge(root);
+      } else {
+        throw std::runtime_error("operation not supported " + operation);
+      }
+    }
+  }
+
+  bool handlePost(CivetServer *server, struct mg_connection *conn) {
 
 Review comment:
   unused parameter 'server'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r397588367
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @bakaid Your analysis of the crash is spot on. I could also reproduce the C2JstackTest crash on Linux and MacOSX.
   The reason behind the crash stems from the memory leak due to shared_ptr cycles. When the test completes the FlowController is never destroyed and that means the thread pool is not shutdown. Therefore, the C2Agent thread or any other thread that accesses the static memory that was released by the main thread causes a segfault.
   
   Solutions: There are different approaches to solve this problem.
   1. Do not share the threadpool with C2Agent. Run the C2Agent operations in a separate thread and when we stop FlowController we can optionally stop the C2Agent as well.
   
   2. Optionally stop the threadpool in FlowController::stop method. We accept boolean parameter to determine shutting down the threadpool.
    
   3. Expose a public member function FlowController::stopThreadPool that will just shutdown the threadpool.
   
   I personally like option 1, but I would like to create a separate Jira to do that as it will be more extensive work that well surpasses the scope of this PR.
   
   Option 2 is also acceptable but there is a caveat while trying to stop components other than FlowController. It requires more investigation finding its side effects.
   
   Option 1 seems to be a cleaner approach and so effort and time spent for option 2 makes little sense.
   In the scope of this PR, I will go with option 3 and here is a follow up Jira that will implement Option 1 and get rid of the extra method we have to add here.
   
   Follow up: https://issues.apache.org/jira/browse/MINIFICPP-1186
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389004074
 
 

 ##########
 File path: libminifi/include/utils/ThreadPool.h
 ##########
 @@ -412,6 +414,22 @@ class ThreadPool {
   void run_tasks(std::shared_ptr<WorkerThread> thread);
 };
 
+template<typename T>
+bool ThreadPool<T>::execute(Worker<T> &&task) {
 
 Review comment:
   Reverted this change.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395845086
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 you are right, the proper calls to the IdGenerator was added in this PR. Even with the previous version though, `CoreComponent::id_generator_` was statically initialized (see the original `Core.cpp`, so an issue with it not yet being initialized could only have occured if some other static initialization tried to use. Even then, it should have failed much sooner, not at the underlying uuid_impl_ point: the `id_generator_` shared_ptr itself would have been nullptr.
   
   Anyway, unfortunately I was able to reproduce the issue on Linux with your latest version. The IdGenerator (or the underlying uuid_impl_ ) is not nullptr, rather, a substructure of the uuid object, the md5 calculator seems to be changed/overwritten at some time:
   ```
   Thread 58 "C2JstackTest" received signal SIGSEGV, Segmentation fault.
   [Switching to Thread 0x7ffff4f07700 (LWP 39351)]
   MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   176	    idx = (unsigned int)((context->count[0] >> 3) & 0x3F);
   (gdb) bt
   #0  MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   #1  0x0000555555952abe in uuid_md5_update (md5=<optimized out>, data_ptr=data_ptr@entry=0x7ffff4f06040, data_len=data_len@entry=32) at uuid_md5.c:398
   #2  0x00005555559537a3 in uuid_prng_data (prng=0x555555df8c80, data_ptr=data_ptr@entry=0x7ffff4f060c8, data_len=data_len@entry=2) at uuid_prng.c:164
   #3  0x0000555555951e12 in uuid_make_v1 (ap=<optimized out>, mode=<optimized out>, uuid=<optimized out>) at uuid.c:952
   #4  uuid_make (uuid=0x555555df8030, mode=1) at uuid.c:1181
   #5  0x000055555595037a in uuid::make(unsigned int, ...) ()
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   #7  0x00005555557dd255 in org::apache::nifi::minifi::utils::IdGenerator::generate (this=<optimized out>, ident=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:329
   #8  0x0000555555770084 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent (name=..., this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/Core.h:166
   #9  org::apache::nifi::minifi::core::Connectable::Connectable (this=0x7ffff4f06430, name=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/core/Connectable.cpp:43
   #10 0x0000555555816de8 in org::apache::nifi::minifi::state::response::ResponseNode::ResponseNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:48
   #11 org::apache::nifi::minifi::state::response::DeviceInformation::DeviceInformation (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:97
   #12 org::apache::nifi::minifi::state::response::StateMonitorNode::StateMonitorNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/StateMonitor.h:53
   #13 org::apache::nifi::minifi::state::response::AgentStatus::AgentStatus (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:418
   #14 org::apache::nifi::minifi::state::response::AgentNode::getAgentStatus (this=this@entry=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:687
   #15 0x000055555581d5bf in org::apache::nifi::minifi::state::response::AgentInformation::serialize (this=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:738
   #16 0x000055555573ed5e in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat (this=0x5555560199c0) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:314
   #17 0x000055555573f14f in org::apache::nifi::minifi::c2::C2Agent::<lambda()>::operator()(void) const (__closure=0x7ffff4f06d10) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:100
   #18 0x000055555573f951 in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const (this=0x7ffff4f06d10) at /usr/include/c++/7/bits/std_function.h:706
   #19 org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run (this=0x7ffff4f06ce0) at /home/bakaid/nifi-minifi-cpp/libminifi/include/utils/ThreadPool.h:97
   #20 0x00005555557f2ec9 in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run_tasks (this=0x555556012aa8, thread=...)
       at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/ThreadPool.cpp:49
   #21 0x00005555557e3d49 in std::__invoke_impl<void, void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__t=<optimized out>, __f=<optimized out>)
       at /usr/include/c++/7/bits/invoke.h:73
   #22 std::__invoke<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__fn=<optimized out>) at /usr/include/c++/7/bits/invoke.h:95
   #23 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::__call<void, , 0ul, 1ul>(std::tuple<>&&, std::_Index_tuple<0ul, 1ul>) (__args=..., this=<optimized out>)
       at /usr/include/c++/7/functional:467
   #24 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::operator()<, void>() (this=<optimized out>) at /usr/include/c++/7/functional:551
   #25 std::_Function_handler<void (), std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)> >::_M_invoke(std::_Any_data const&) (__functor=...)
       at /usr/include/c++/7/bits/std_function.h:316
   #26 0x00007ffff74c96ef in execute_native_thread_routine () at ../../../../../src/libstdc++-v3/src/c++11/thread.cc:80
   #27 0x00007ffff7bbd6db in start_thread (arg=0x7ffff4f07700) at pthread_create.c:463
   #28 0x00007ffff6f2488f in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95
   (gdb) f 6
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   286	    uuid_impl_->make(mode);
   (gdb) p *uuid_impl_.get()->ctx
   $13 = {obj = {time_low = 3412410872, time_mid = 27348, time_hi_and_version = 490, clock_seq_hi_and_reserved = 33 '!', clock_seq_low = 0 '\000', node = "\000\000\000\000\000"}, prng = 0x555555df8c80, md5 = 0x555555df7e70, sha1 = 0x0,
     mac = "\241\000\000\000\000", time_last = {tv_sec = 93825001311728, tv_usec = 93825001663376}, time_seq = 0}
   (gdb) p *uuid_impl_.get()->ctx->prng
   $14 = {dev = 1330905088, md5 = 0x21, cnt = 93825001295265}
   (gdb) p *uuid_impl_.get()->ctx->prng->md5
   Cannot access memory at address 0x21
   ```
   It is nondeterministic, and there are also random crashes and aborts at other points.
   valgrind also sometimes finds errors:
   ```
   ==38778== HEAP SUMMARY:
   ==38778==     in use at exit: 1,887,773 bytes in 17,209 blocks
   ==38778==   total heap usage: 98,643 allocs, 81,434 frees, 16,173,074 bytes allocated
   ==38778==
   ==38778== Searching for pointers to 17,209 not-freed blocks
   ==38778== Checked 60,375,896 bytes
   ==38778==
   ==38778== LEAK SUMMARY:
   ==38778==    definitely lost: 1,088 bytes in 1 blocks
   ==38778==    indirectly lost: 19,129 bytes in 126 blocks
   ==38778==      possibly lost: 2,128 bytes in 7 blocks
   ==38778==    still reachable: 1,865,428 bytes in 17,075 blocks
   ==38778==                       of which reachable via heuristic:
   ==38778==                         multipleinheritance: 744 bytes in 1 blocks
   ==38778==         suppressed: 0 bytes in 0 blocks
   ==38778== Rerun with --leak-check=full to see details of leaked memory
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ==38778==
   ==38778== 425 errors in context 1 of 1:
   ==38778== Conditional jump or move depends on uninitialised value(s)
   ==38778==    at 0x4C32D08: strlen (in /usr/lib/valgrind/vgpreload_memcheck-amd64-linux.so)
   ==38778==    by 0x2ACC57: length (char_traits.h:320)
   ==38778==    by 0x2ACC57: std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >::basic_string(char const*, std::allocator<char> const&) [clone .isra.565] (basic_string.h:511)
   ==38778==    by 0x2D5D2B: readPost (HTTPHandlers.h:360)
   ==38778==    by 0x2D5D2B: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:421)
   ==38778==    by 0x2D6640: HeartbeatHandler::handlePost(CivetServer*, mg_connection*) (HTTPHandlers.h:438)
   ==38778==    by 0x4AB8C8: CivetServer::requestHandler(mg_connection*, void*) (CivetServer.cpp:148)
   ==38778==    by 0x4B9E4C: handle_request (civetweb.c:12704)
   ==38778==    by 0x4BB4DC: process_new_connection (civetweb.c:15918)
   ==38778==    by 0x4BB82F: worker_thread_run (civetweb.c:16225)
   ==38778==    by 0x4BB82F: worker_thread (civetweb.c:16290)
   ==38778==    by 0x4E436DA: start_thread (pthread_create.c:463)
   ==38778==    by 0x5B3E88E: clone (clone.S:95)
   ==38778==  Uninitialised value was created by a stack allocation
   ==38778==    at 0x2D5CB0: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:420)
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ```
   
   The whole thing seems to me like a memory corruption/stack corruption issue. I am investigating it further, but I do not at all find it unlikely that it could be caused by the things we do in the SIGUSR2 signal handlers to get BackTraces.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389968278
 
 

 ##########
 File path: libminifi/include/c2/C2Agent.h
 ##########
 @@ -244,6 +239,10 @@ class C2Agent : public state::UpdateController, public state::response::Response
   std::string bin_location_;
 
   std::shared_ptr<logging::Logger> logger_;
+
+  utils::ThreadPool<state::Update> heartbeat_thread_pool_;
 
 Review comment:
   I'm afraid it's going to conflict with my threadpool PR, which is about having one single threadpool.
   In case that gets merged before this, let's have a talk to fit this in that architecture. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r388296214
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -916,57 +916,37 @@ int16_t FlowController::clearConnection(const std::string &connection) {
   return -1;
 }
 
-int16_t FlowController::getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
+std::shared_ptr<state::response::ResponseNode> FlowController::getMetricsNode() const {
   std::lock_guard<std::mutex> lock(metrics_mutex_);
-
-  for (auto metric : root_response_nodes_) {
-    metric_vector.push_back(metric.second);
+  const auto iter = root_response_nodes_.find("metrics");
+  if (iter != root_response_nodes_.end()) {
+    return iter->second;
   }
-
-  return 0;
+  return nullptr;
 }
 
-int16_t FlowController::getMetricsNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
-  std::lock_guard<std::mutex> lock(metrics_mutex_);
-  if (metricsClass == 0) {
-    for (auto metric : device_information_) {
-      metric_vector.push_back(metric.second);
-    }
-  } else {
-    auto metrics = component_metrics_by_id_[metricsClass];
-    for (const auto &metric : metrics) {
-      metric_vector.push_back(metric);
-    }
-  }
-  return 0;
-}
+std::vector<std::shared_ptr<state::response::ResponseNode>> FlowController::getHeartbeatNodes(bool includeManifest) const {
+  std::string fullHb{"true"};
+  configuration_->get("nifi.c2.full.heartbeat", fullHb);
 
 Review comment:
   In some ideal future state I would be more happy if the ResponseNodes themselves would be initialized with the configuration and could decide themselves what to based on it, but I do think this is a big step in the right direction, as it eliminates the class duplication and makes "communication" with the ResponseNode possible.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395845086
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 you are right, the proper calls to the IdGenerator was added in this PR. Even with the previous version though, `CoreComponent::id_generator_` was statically initialized (see the original `Core.cpp`), so an issue with it not yet being initialized could only have occured if some other static initialization tried to use it. Even then, it should have failed much sooner, not at the underlying uuid_impl_ point: the `id_generator_` shared_ptr itself would have been nullptr.
   
   Anyway, unfortunately I was able to reproduce the issue on Linux with your latest version. The IdGenerator (or the underlying uuid_impl_ ) is not nullptr, rather, a substructure of the uuid object, the md5 calculator seems to be changed/overwritten at some time:
   ```
   Thread 58 "C2JstackTest" received signal SIGSEGV, Segmentation fault.
   [Switching to Thread 0x7ffff4f07700 (LWP 39351)]
   MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   176	    idx = (unsigned int)((context->count[0] >> 3) & 0x3F);
   (gdb) bt
   #0  MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   #1  0x0000555555952abe in uuid_md5_update (md5=<optimized out>, data_ptr=data_ptr@entry=0x7ffff4f06040, data_len=data_len@entry=32) at uuid_md5.c:398
   #2  0x00005555559537a3 in uuid_prng_data (prng=0x555555df8c80, data_ptr=data_ptr@entry=0x7ffff4f060c8, data_len=data_len@entry=2) at uuid_prng.c:164
   #3  0x0000555555951e12 in uuid_make_v1 (ap=<optimized out>, mode=<optimized out>, uuid=<optimized out>) at uuid.c:952
   #4  uuid_make (uuid=0x555555df8030, mode=1) at uuid.c:1181
   #5  0x000055555595037a in uuid::make(unsigned int, ...) ()
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   #7  0x00005555557dd255 in org::apache::nifi::minifi::utils::IdGenerator::generate (this=<optimized out>, ident=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:329
   #8  0x0000555555770084 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent (name=..., this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/Core.h:166
   #9  org::apache::nifi::minifi::core::Connectable::Connectable (this=0x7ffff4f06430, name=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/core/Connectable.cpp:43
   #10 0x0000555555816de8 in org::apache::nifi::minifi::state::response::ResponseNode::ResponseNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:48
   #11 org::apache::nifi::minifi::state::response::DeviceInformation::DeviceInformation (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:97
   #12 org::apache::nifi::minifi::state::response::StateMonitorNode::StateMonitorNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/StateMonitor.h:53
   #13 org::apache::nifi::minifi::state::response::AgentStatus::AgentStatus (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:418
   #14 org::apache::nifi::minifi::state::response::AgentNode::getAgentStatus (this=this@entry=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:687
   #15 0x000055555581d5bf in org::apache::nifi::minifi::state::response::AgentInformation::serialize (this=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:738
   #16 0x000055555573ed5e in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat (this=0x5555560199c0) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:314
   #17 0x000055555573f14f in org::apache::nifi::minifi::c2::C2Agent::<lambda()>::operator()(void) const (__closure=0x7ffff4f06d10) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:100
   #18 0x000055555573f951 in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const (this=0x7ffff4f06d10) at /usr/include/c++/7/bits/std_function.h:706
   #19 org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run (this=0x7ffff4f06ce0) at /home/bakaid/nifi-minifi-cpp/libminifi/include/utils/ThreadPool.h:97
   #20 0x00005555557f2ec9 in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run_tasks (this=0x555556012aa8, thread=...)
       at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/ThreadPool.cpp:49
   #21 0x00005555557e3d49 in std::__invoke_impl<void, void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__t=<optimized out>, __f=<optimized out>)
       at /usr/include/c++/7/bits/invoke.h:73
   #22 std::__invoke<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__fn=<optimized out>) at /usr/include/c++/7/bits/invoke.h:95
   #23 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::__call<void, , 0ul, 1ul>(std::tuple<>&&, std::_Index_tuple<0ul, 1ul>) (__args=..., this=<optimized out>)
       at /usr/include/c++/7/functional:467
   #24 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::operator()<, void>() (this=<optimized out>) at /usr/include/c++/7/functional:551
   #25 std::_Function_handler<void (), std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)> >::_M_invoke(std::_Any_data const&) (__functor=...)
       at /usr/include/c++/7/bits/std_function.h:316
   #26 0x00007ffff74c96ef in execute_native_thread_routine () at ../../../../../src/libstdc++-v3/src/c++11/thread.cc:80
   #27 0x00007ffff7bbd6db in start_thread (arg=0x7ffff4f07700) at pthread_create.c:463
   #28 0x00007ffff6f2488f in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95
   (gdb) f 6
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   286	    uuid_impl_->make(mode);
   (gdb) p *uuid_impl_.get()->ctx
   $13 = {obj = {time_low = 3412410872, time_mid = 27348, time_hi_and_version = 490, clock_seq_hi_and_reserved = 33 '!', clock_seq_low = 0 '\000', node = "\000\000\000\000\000"}, prng = 0x555555df8c80, md5 = 0x555555df7e70, sha1 = 0x0,
     mac = "\241\000\000\000\000", time_last = {tv_sec = 93825001311728, tv_usec = 93825001663376}, time_seq = 0}
   (gdb) p *uuid_impl_.get()->ctx->prng
   $14 = {dev = 1330905088, md5 = 0x21, cnt = 93825001295265}
   (gdb) p *uuid_impl_.get()->ctx->prng->md5
   Cannot access memory at address 0x21
   ```
   It is nondeterministic, and there are also random crashes and aborts at other points.
   valgrind also sometimes finds errors:
   ```
   ==38778== HEAP SUMMARY:
   ==38778==     in use at exit: 1,887,773 bytes in 17,209 blocks
   ==38778==   total heap usage: 98,643 allocs, 81,434 frees, 16,173,074 bytes allocated
   ==38778==
   ==38778== Searching for pointers to 17,209 not-freed blocks
   ==38778== Checked 60,375,896 bytes
   ==38778==
   ==38778== LEAK SUMMARY:
   ==38778==    definitely lost: 1,088 bytes in 1 blocks
   ==38778==    indirectly lost: 19,129 bytes in 126 blocks
   ==38778==      possibly lost: 2,128 bytes in 7 blocks
   ==38778==    still reachable: 1,865,428 bytes in 17,075 blocks
   ==38778==                       of which reachable via heuristic:
   ==38778==                         multipleinheritance: 744 bytes in 1 blocks
   ==38778==         suppressed: 0 bytes in 0 blocks
   ==38778== Rerun with --leak-check=full to see details of leaked memory
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ==38778==
   ==38778== 425 errors in context 1 of 1:
   ==38778== Conditional jump or move depends on uninitialised value(s)
   ==38778==    at 0x4C32D08: strlen (in /usr/lib/valgrind/vgpreload_memcheck-amd64-linux.so)
   ==38778==    by 0x2ACC57: length (char_traits.h:320)
   ==38778==    by 0x2ACC57: std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >::basic_string(char const*, std::allocator<char> const&) [clone .isra.565] (basic_string.h:511)
   ==38778==    by 0x2D5D2B: readPost (HTTPHandlers.h:360)
   ==38778==    by 0x2D5D2B: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:421)
   ==38778==    by 0x2D6640: HeartbeatHandler::handlePost(CivetServer*, mg_connection*) (HTTPHandlers.h:438)
   ==38778==    by 0x4AB8C8: CivetServer::requestHandler(mg_connection*, void*) (CivetServer.cpp:148)
   ==38778==    by 0x4B9E4C: handle_request (civetweb.c:12704)
   ==38778==    by 0x4BB4DC: process_new_connection (civetweb.c:15918)
   ==38778==    by 0x4BB82F: worker_thread_run (civetweb.c:16225)
   ==38778==    by 0x4BB82F: worker_thread (civetweb.c:16290)
   ==38778==    by 0x4E436DA: start_thread (pthread_create.c:463)
   ==38778==    by 0x5B3E88E: clone (clone.S:95)
   ==38778==  Uninitialised value was created by a stack allocation
   ==38778==    at 0x2D5CB0: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:420)
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ```
   
   The whole thing seems to me like a memory corruption/stack corruption issue. I am investigating it further, but I do not at all find it unlikely that it could be caused by the things we do in the SIGUSR2 signal handlers to get BackTraces.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392360339
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -907,29 +908,44 @@ int16_t FlowController::clearConnection(const std::string &connection) {
   return -1;
 }
 
-int16_t FlowController::getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
+std::shared_ptr<state::response::ResponseNode> FlowController::getMetricsNode(const std::string& metricsClass) const {
   std::lock_guard<std::mutex> lock(metrics_mutex_);
-
-  for (auto metric : root_response_nodes_) {
-    metric_vector.push_back(metric.second);
+  if (!metricsClass.empty()) {
+    const auto citer = component_metrics_.find(metricsClass);
+    if (citer != component_metrics_.end()) {
+      return citer->second;
+    }
+  } else {
+    const auto iter = root_response_nodes_.find("metrics");
+    if (iter != root_response_nodes_.end()) {
+      return iter->second;
+    }
 
 Review comment:
   Indeed, I missed that. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392202093
 
 

 ##########
 File path: extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
 ##########
 @@ -50,98 +50,45 @@
 #include "protocols/RESTReceiver.h"
 #include "protocols/RESTSender.h"
 #include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 #include "agent/build_description.h"
 #include "processors/LogAttribute.h"
 
-class Responder : public CivetHandler {
+class LightWeightC2Handler : public HeartbeatHandler {
  public:
-  explicit Responder(bool isSecure)
-      : isSecure(isSecure) {
+  explicit LightWeightC2Handler(bool isSecure)
+      : HeartbeatHandler(isSecure),
+        calls_(0) {
   }
 
-  std::string readPost(struct mg_connection *conn) {
-    std::string response;
-    int blockSize = 1024 * sizeof(char), readBytes;
+  virtual ~LightWeightC2Handler() = default;
 
-    char buffer[1024];
-    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
-      response.append(buffer, 0, (readBytes / sizeof(char)));
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn)  {
+    (void)conn;
+    if (calls_ == 0) {
+      verifyJsonHasAgentManifest(root);
+    } else {
+      assert(root.HasMember("agentInfo") == true);
+      assert(root["agentInfo"].HasMember("agentManifest") == false);
     }
-    return response;
+    calls_++;
   }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    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;
-      std::string 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",
-              resp.length());
-    mg_printf(conn, "%s", resp.c_str());
-    return true;
-  }
-
- protected:
-  bool isSecure;
+ private:
+  std::atomic<size_t> calls_;
 
 Review comment:
   C.48: Prefer in-class initializers to member initializers in constructors for constant initializers
   
   http://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#c48-prefer-in-class-initializers-to-member-initializers-in-constructors-for-constant-initializers

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390669052
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
 
 Review comment:
   Not sure why, but makes sense to simplify this. We are just reading 1024 bytes from the connection in each call. The variable blockSize is extraneous.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r409396297
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -89,44 +93,68 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
         }
         request_mutex.unlock();
       }
-
-      if ( time_since > heart_beat_period_ ) {
-        last_run_ = now;
-        try {
-          performHeartBeat();
-        }
-        catch(const std::exception &e) {
-          logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
-        }
-        catch(...) {
-          logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
-        }
+      try {
+        performHeartBeat();
+      }
+      catch(const std::exception &e) {
+        logger_->log_error("Exception occurred while performing heartbeat. error: %s", e.what());
+      }
+      catch(...) {
+        logger_->log_error("Unknonwn exception occurred while performing heartbeat.");
       }
 
       checkTriggers();
 
-      std::this_thread::sleep_for(std::chrono::milliseconds(heart_beat_period_ > 500 ? 500 : heart_beat_period_));
-      return state::Update(state::UpdateStatus(state::UpdateState::READ_COMPLETE, false));
+      return utils::TaskRescheduleInfo::RetryIn(std::chrono::milliseconds(heart_beat_period_));
     };
-
   functions_.push_back(c2_producer_);
 
   c2_consumer_ = [&]() {
     auto now = std::chrono::steady_clock::now();
     if ( queue_mutex.try_lock_until(now + std::chrono::seconds(1)) ) {
-      if (responses.size() > 0) {
-        const C2Payload payload(std::move(responses.back()));
-        responses.pop_back();
-        extractPayload(std::move(payload));
+      if (responses.empty()) {
+        queue_mutex.unlock();
+        return utils::TaskRescheduleInfo::RetryImmediately();
 
 Review comment:
   RetryImmediately was introduced for a reason: when something takes quite long (an ontrigger call of a processor for eg), it might happen that the next schedule is already in the past. Although we cannot continue executing it as that would result in starvation. 
   In this case it's fair to reschedule immediately. 
   This means that the given task is put back to the end of the queue, so in case there are tasks already waiting for CPU, those are executed first, otherwise this one can be continued immediately without unnecessary wait. 
   
   In this current aspect, you are right, it shouldn't be used for polling something, but otherwise it's something we need.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392181842
 
 

 ##########
 File path: C2.md
 ##########
 @@ -49,6 +49,11 @@ Release 0.6.0: Please note that all c2 properties now exist as `nifi.c2.*`. If y
 files contain the former naming convention of `c2.*`, we will continue to support that as
 an alternate key, but you are encouraged to switch your configuration options as soon as possible.
 
+Note: In release 0.8.0 there is a configuration option to minizime the heartbeat payload size by excluding agent manifest.
 
 Review comment:
   typo: s/minizime/minimize/

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393355142
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   I guess that's a timing issue: in case the c2 component is initialized before the static id generator, it can crash. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389963074
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPIntegrationBase.h
 ##########
 @@ -91,4 +87,64 @@ void CoapIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   }
 }
 
+class VerifyC2Base : public CoapIntegrationBase {
+ public:
+  explicit VerifyC2Base(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  virtual void testSetup() {
+    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setDebug<LogTestController>();
+  }
+
+  void runAssertions() {
 
 Review comment:
   This should be defined by any derived class (to make sure they test something), so I think you can delete it.
   That has the advantage of making this baseclass abstract.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r388244326
 
 

 ##########
 File path: libminifi/include/properties/Configure.h
 ##########
 @@ -36,10 +36,21 @@ class Configure : public Properties {
     std::lock_guard<std::mutex> lock(mutex_);
     agent_identifier_ = identifier;
   }
-  std::string getAgentIdentifier() {
+  std::string getAgentIdentifier() const {
     std::lock_guard<std::mutex> lock(mutex_);
     return agent_identifier_;
   }
+
+  void setAgentClass(const std::string& agentClass) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    agent_class_ = agentClass;
+  }
+
+  std::string getAgentClass() const {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return agent_class_;
+  }
+
 
 Review comment:
   New configuration options (`nifi.c2.full.heartbeat`) should be added to `Configure.h` and `Configure.cpp` as `static const char *` members, like the rest.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392200840
 
 

 ##########
 File path: extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
 ##########
 @@ -50,98 +50,45 @@
 #include "protocols/RESTReceiver.h"
 #include "protocols/RESTSender.h"
 #include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 #include "agent/build_description.h"
 #include "processors/LogAttribute.h"
 
-class Responder : public CivetHandler {
+class LightWeightC2Handler : public HeartbeatHandler {
  public:
-  explicit Responder(bool isSecure)
-      : isSecure(isSecure) {
+  explicit LightWeightC2Handler(bool isSecure)
+      : HeartbeatHandler(isSecure),
+        calls_(0) {
   }
 
-  std::string readPost(struct mg_connection *conn) {
-    std::string response;
-    int blockSize = 1024 * sizeof(char), readBytes;
+  virtual ~LightWeightC2Handler() = default;
 
-    char buffer[1024];
-    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
-      response.append(buffer, 0, (readBytes / sizeof(char)));
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn)  {
+    (void)conn;
 
 Review comment:
   You can avoid unused warnings by not naming the argument.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r397589151
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -32,24 +32,34 @@
 #include "utils/file/FileUtils.h"
 #include "utils/file/FileManager.h"
 #include "utils/HTTPClient.h"
+#include "utils/GeneralUtils.h"
+#include "utils/Monitors.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace c2 {
 
-C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<state::StateMonitor> &updateSink,
-                 const std::shared_ptr<Configure> &configuration)
+std::shared_ptr<utils::IdGenerator> C2Agent::id_generator_ = utils::IdGenerator::getIdGenerator();
 
 Review comment:
   Yep, that remained to be fixed. I forgot to make correction here. There is one more in FlowController.cpp.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r388337240
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -449,13 +451,9 @@ void FlowController::initializeC2() {
 
   if (!c2_initialized_) {
     configuration_->setAgentIdentifier(identifier_str);
-    state::StateManager::initialize();
-    std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+    c2_agent_ = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
 
 Review comment:
   This was and still is a `shared_ptr` cycle between `C2Agent` and `FlowController`, which is a bad practice and the reason for many of our memory leaks.
   Since we are refactoring this we should solve this as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r397762023
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 Currently on master C2 has its own thread pool (StateManager's `listener_thread_pool_`), which is separate from the processor thread pool.
   As I wrote in an another review comment, using the same thread pool for both C2 and processors is not a good idea.
   Merging this PR with the unified thread pool would actually make things worse compared to master, so **I am -1 on this PR** until this is changed.
   Therefore, because the thread pool separation should be done anyway, please solve this issue with `Option 1`.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r395845086
 
 

 ##########
 File path: extensions/http-curl/tests/C2JstackTest.cpp
 ##########
 @@ -16,152 +16,63 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
 #include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "c2/C2Agent.h"
-#include "CivetServer.h"
-#include <cstring>
-#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-
-class ConfigHandler : public CivetHandler {
+class VerifyC2DescribeJstack : public VerifyC2Describe {
  public:
-  ConfigHandler() {
-    calls_ = 0;
-  }
-  bool handlePost(CivetServer *server, struct mg_connection *conn) {
-    calls_++;
-    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
-          "\"operation\" : \"describe\", "
-          "\"operationid\" : \"8675309\", "
-          "\"name\": \"jstack\""
-          "}]}";
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                heartbeat_response.length());
-      mg_printf(conn, "%s", heartbeat_response.c_str());
-
-
-    return true;
+  explicit VerifyC2DescribeJstack(bool isSecure)
+      : VerifyC2Describe(isSecure) {
   }
 
-  bool handleGet(CivetServer *server, struct mg_connection *conn) {
-    std::ifstream myfile(test_file_location_.c_str());
-
-    if (myfile.is_open()) {
-      std::stringstream buffer;
-      buffer << myfile.rdbuf();
-      std::string str = buffer.str();
-      myfile.close();
-      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-                str.length());
-      mg_printf(conn, "%s", str.c_str());
-    } else {
-      mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
-    }
-
-    return true;
+  virtual void runAssertions() {
+    assert(LogTestController::getInstance().contains("SchedulingAgent") == true);
   }
-  std::string test_file_location_;
-  std::atomic<size_t> calls_;
 };
 
-int main(int argc, char **argv) {
-  mg_init_library(0);
-  LogTestController::getInstance().setInfo<minifi::FlowController>();
-  LogTestController::getInstance().setDebug<minifi::utils::HTTPClient>();
-  LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
-  LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+class DescribeJstackHandler : public HeartbeatHandler {
+ public:
+  explicit DescribeJstackHandler(bool isSecure)
+     : HeartbeatHandler(isSecure) {
+  }
 
-  const char *options[] = { "document_root", ".", "listening_ports", "0", 0 };
-  std::vector<std::string> cpp_options;
-  for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
-    cpp_options.push_back(options[i]);
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    sendHeartbeatResponse("DESCRIBE", "jstack", "889398", conn);
   }
 
-  CivetServer server(cpp_options);
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+    assert(root.HasMember("Flowcontroller threadpool #0") == true);
+  }
 
-  std::string port_str = std::to_string(server.getListeningPorts()[0]);
+};
 
-  ConfigHandler h_ex;
-  server.addHandler("/update", h_ex);
-  std::string key_dir, test_file_location;
+int main(int argc, char **argv) {
+  std::string key_dir, test_file_location, url;
+  url = "http://localhost:0/api/heartbeat";
   if (argc > 1) {
-    h_ex.test_file_location_ = test_file_location = argv[1];
-    key_dir = argv[2];
+    test_file_location = argv[1];
+    if (argc > 2) {
+      url = "https://localhost:0/api/heartbeat";
+      key_dir = argv[2];
+    }
   }
 
+  bool isSecure = false;
+  if (url.find("https") != std::string::npos) {
+    isSecure = true;
+  }
 
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::string c2_rest_url = "http://localhost:" + port_str + "/update";
-
-  configuration->set("c2.rest.url", c2_rest_url);
-  configuration->set("c2.agent.heartbeat.period", "1000");
-
-  std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
-  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location));
-  std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
-  true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location);
+  VerifyC2DescribeJstack harness(isSecure);
 
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(ptr.get());
-  ptr.release();
-  auto start = std::chrono::system_clock::now();
+  harness.setKeyDir(key_dir);
 
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
+  DescribeJstackHandler responder(isSecure);
 
-  controller->waitUnload(60000);
-  auto then = std::chrono::system_clock::now();
+  harness.setUrl(url, &responder);
 
-  auto milliseconds = std::chrono::duration_cast<std::chrono::milliseconds>(then - start).count();
-  std::string logs = LogTestController::getInstance().log_output.str();
-  #ifndef WIN32
-  assert(logs.find("SchedulingAgent") != std::string::npos);
-  #endif
-  LogTestController::getInstance().reset();
-  assert(h_ex.calls_ <= (milliseconds / 1000) + 1);
+  harness.run(test_file_location);
 
-  return 0;
 }
 
 Review comment:
   @msharee9 you are right, the proper calls to the IdGenerator was added in this PR. Even with the previous version though, `CoreComponent::id_generator_` was statically initialized (see the original `Core.cpp`), so an issue with it not yet being initialized could only have occured if some other static initialization tried to use. Even then, it should have failed much sooner, not at the underlying uuid_impl_ point: the `id_generator_` shared_ptr itself would have been nullptr.
   
   Anyway, unfortunately I was able to reproduce the issue on Linux with your latest version. The IdGenerator (or the underlying uuid_impl_ ) is not nullptr, rather, a substructure of the uuid object, the md5 calculator seems to be changed/overwritten at some time:
   ```
   Thread 58 "C2JstackTest" received signal SIGSEGV, Segmentation fault.
   [Switching to Thread 0x7ffff4f07700 (LWP 39351)]
   MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   176	    idx = (unsigned int)((context->count[0] >> 3) & 0x3F);
   (gdb) bt
   #0  MD5Update (context=0x21, input=0x7ffff4f06040 "\354\004u^", inputLen=32) at uuid_md5.c:176
   #1  0x0000555555952abe in uuid_md5_update (md5=<optimized out>, data_ptr=data_ptr@entry=0x7ffff4f06040, data_len=data_len@entry=32) at uuid_md5.c:398
   #2  0x00005555559537a3 in uuid_prng_data (prng=0x555555df8c80, data_ptr=data_ptr@entry=0x7ffff4f060c8, data_len=data_len@entry=2) at uuid_prng.c:164
   #3  0x0000555555951e12 in uuid_make_v1 (ap=<optimized out>, mode=<optimized out>, uuid=<optimized out>) at uuid.c:952
   #4  uuid_make (uuid=0x555555df8030, mode=1) at uuid.c:1181
   #5  0x000055555595037a in uuid::make(unsigned int, ...) ()
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   #7  0x00005555557dd255 in org::apache::nifi::minifi::utils::IdGenerator::generate (this=<optimized out>, ident=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:329
   #8  0x0000555555770084 in org::apache::nifi::minifi::core::CoreComponent::CoreComponent (name=..., this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/Core.h:166
   #9  org::apache::nifi::minifi::core::Connectable::Connectable (this=0x7ffff4f06430, name=...) at /home/bakaid/nifi-minifi-cpp/libminifi/src/core/Connectable.cpp:43
   #10 0x0000555555816de8 in org::apache::nifi::minifi::state::response::ResponseNode::ResponseNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:48
   #11 org::apache::nifi::minifi::state::response::DeviceInformation::DeviceInformation (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/MetricsBase.h:97
   #12 org::apache::nifi::minifi::state::response::StateMonitorNode::StateMonitorNode (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/../nodes/StateMonitor.h:53
   #13 org::apache::nifi::minifi::state::response::AgentStatus::AgentStatus (name="status", this=0x7ffff4f06430) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:418
   #14 org::apache::nifi::minifi::state::response::AgentNode::getAgentStatus (this=this@entry=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:687
   #15 0x000055555581d5bf in org::apache::nifi::minifi::state::response::AgentInformation::serialize (this=0x555556019640) at /home/bakaid/nifi-minifi-cpp/libminifi/include/core/state/nodes/AgentInformation.h:738
   #16 0x000055555573ed5e in org::apache::nifi::minifi::c2::C2Agent::performHeartBeat (this=0x5555560199c0) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:314
   #17 0x000055555573f14f in org::apache::nifi::minifi::c2::C2Agent::<lambda()>::operator()(void) const (__closure=0x7ffff4f06d10) at /home/bakaid/nifi-minifi-cpp/libminifi/src/c2/C2Agent.cpp:100
   #18 0x000055555573f951 in std::function<org::apache::nifi::minifi::utils::TaskRescheduleInfo ()>::operator()() const (this=0x7ffff4f06d10) at /usr/include/c++/7/bits/std_function.h:706
   #19 org::apache::nifi::minifi::utils::Worker<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run (this=0x7ffff4f06ce0) at /home/bakaid/nifi-minifi-cpp/libminifi/include/utils/ThreadPool.h:97
   #20 0x00005555557f2ec9 in org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::run_tasks (this=0x555556012aa8, thread=...)
       at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/ThreadPool.cpp:49
   #21 0x00005555557e3d49 in std::__invoke_impl<void, void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__t=<optimized out>, __f=<optimized out>)
       at /usr/include/c++/7/bits/invoke.h:73
   #22 std::__invoke<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*&)(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>), org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*&, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>&> (__fn=<optimized out>) at /usr/include/c++/7/bits/invoke.h:95
   #23 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::__call<void, , 0ul, 1ul>(std::tuple<>&&, std::_Index_tuple<0ul, 1ul>) (__args=..., this=<optimized out>)
       at /usr/include/c++/7/functional:467
   #24 std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)>::operator()<, void>() (this=<optimized out>) at /usr/include/c++/7/functional:551
   #25 std::_Function_handler<void (), std::_Bind<void (org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>::*(org::apache::nifi::minifi::utils::ThreadPool<org::apache::nifi::minifi::utils::TaskRescheduleInfo>*, std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>))(std::shared_ptr<org::apache::nifi::minifi::utils::WorkerThread>)> >::_M_invoke(std::_Any_data const&) (__functor=...)
       at /usr/include/c++/7/bits/std_function.h:316
   #26 0x00007ffff74c96ef in execute_native_thread_routine () at ../../../../../src/libstdc++-v3/src/c++11/thread.cc:80
   #27 0x00007ffff7bbd6db in start_thread (arg=0x7ffff4f07700) at pthread_create.c:463
   #28 0x00007ffff6f2488f in clone () at ../sysdeps/unix/sysv/linux/x86_64/clone.S:95
   (gdb) f 6
   #6  0x00005555557dd0f1 in org::apache::nifi::minifi::utils::IdGenerator::generateWithUuidImpl (this=0x555555df5e70, mode=<optimized out>, output=0x7ffff4f06300 "\006") at /home/bakaid/nifi-minifi-cpp/libminifi/src/utils/Id.cpp:286
   286	    uuid_impl_->make(mode);
   (gdb) p *uuid_impl_.get()->ctx
   $13 = {obj = {time_low = 3412410872, time_mid = 27348, time_hi_and_version = 490, clock_seq_hi_and_reserved = 33 '!', clock_seq_low = 0 '\000', node = "\000\000\000\000\000"}, prng = 0x555555df8c80, md5 = 0x555555df7e70, sha1 = 0x0,
     mac = "\241\000\000\000\000", time_last = {tv_sec = 93825001311728, tv_usec = 93825001663376}, time_seq = 0}
   (gdb) p *uuid_impl_.get()->ctx->prng
   $14 = {dev = 1330905088, md5 = 0x21, cnt = 93825001295265}
   (gdb) p *uuid_impl_.get()->ctx->prng->md5
   Cannot access memory at address 0x21
   ```
   It is nondeterministic, and there are also random crashes and aborts at other points.
   valgrind also sometimes finds errors:
   ```
   ==38778== HEAP SUMMARY:
   ==38778==     in use at exit: 1,887,773 bytes in 17,209 blocks
   ==38778==   total heap usage: 98,643 allocs, 81,434 frees, 16,173,074 bytes allocated
   ==38778==
   ==38778== Searching for pointers to 17,209 not-freed blocks
   ==38778== Checked 60,375,896 bytes
   ==38778==
   ==38778== LEAK SUMMARY:
   ==38778==    definitely lost: 1,088 bytes in 1 blocks
   ==38778==    indirectly lost: 19,129 bytes in 126 blocks
   ==38778==      possibly lost: 2,128 bytes in 7 blocks
   ==38778==    still reachable: 1,865,428 bytes in 17,075 blocks
   ==38778==                       of which reachable via heuristic:
   ==38778==                         multipleinheritance: 744 bytes in 1 blocks
   ==38778==         suppressed: 0 bytes in 0 blocks
   ==38778== Rerun with --leak-check=full to see details of leaked memory
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ==38778==
   ==38778== 425 errors in context 1 of 1:
   ==38778== Conditional jump or move depends on uninitialised value(s)
   ==38778==    at 0x4C32D08: strlen (in /usr/lib/valgrind/vgpreload_memcheck-amd64-linux.so)
   ==38778==    by 0x2ACC57: length (char_traits.h:320)
   ==38778==    by 0x2ACC57: std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >::basic_string(char const*, std::allocator<char> const&) [clone .isra.565] (basic_string.h:511)
   ==38778==    by 0x2D5D2B: readPost (HTTPHandlers.h:360)
   ==38778==    by 0x2D5D2B: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:421)
   ==38778==    by 0x2D6640: HeartbeatHandler::handlePost(CivetServer*, mg_connection*) (HTTPHandlers.h:438)
   ==38778==    by 0x4AB8C8: CivetServer::requestHandler(mg_connection*, void*) (CivetServer.cpp:148)
   ==38778==    by 0x4B9E4C: handle_request (civetweb.c:12704)
   ==38778==    by 0x4BB4DC: process_new_connection (civetweb.c:15918)
   ==38778==    by 0x4BB82F: worker_thread_run (civetweb.c:16225)
   ==38778==    by 0x4BB82F: worker_thread (civetweb.c:16290)
   ==38778==    by 0x4E436DA: start_thread (pthread_create.c:463)
   ==38778==    by 0x5B3E88E: clone (clone.S:95)
   ==38778==  Uninitialised value was created by a stack allocation
   ==38778==    at 0x2D5CB0: HeartbeatHandler::verify(mg_connection*) (HTTPHandlers.h:420)
   ==38778==
   ==38778== ERROR SUMMARY: 425 errors from 1 contexts (suppressed: 0 from 0)
   ```
   
   The whole thing seems to me like a memory corruption/stack corruption issue. I am investigating it further, but I do not at all find it unlikely that it could be caused by the things we do in the SIGUSR2 signal handlers to get BackTraces.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392280430
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -66,8 +76,6 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
 
   c2_producer_ = [&]() {
     auto now = std::chrono::steady_clock::now();
 
 Review comment:
   `now` can also be removed by changing from `request_mutex.try_lock_until(now + duration)` to `request_mutex.try_lock_for(duration)` on line 80.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392185357
 
 

 ##########
 File path: extensions/http-curl/tests/C2DescribeManifestTest.cpp
 ##########
 @@ -0,0 +1,91 @@
+/**
+ *
+ * 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.
+ */
+
+#include <sys/stat.h>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <chrono>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <thread>
+#include <type_traits>
+#include <vector>
+#include <iostream>
+#include <sstream>
+#include "HTTPClient.h"
+#include "InvokeHTTP.h"
+#include "TestBase.h"
+#include "utils/StringUtils.h"
+#include "core/Core.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessGroup.h"
+#include "core/yaml/YamlConfiguration.h"
+#include "FlowController.h"
+#include "properties/Configure.h"
+#include "unit/ProvenanceTestHelper.h"
+#include "io/StreamFactory.h"
+#include "c2/C2Agent.h"
+#include "CivetServer.h"
+#include <cstring>
+#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
 
 Review comment:
   We only need a subset of these includes. To name a few, we can throw out `<sys/stat.h>`, `<cassert>`, `<utility>`, `<chrono>`, `<fstream>`, `<memory>`, `<thread>`, `<type_traits>`, `<vector>` and more. Let's not waste vertical space and compilation time!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r388337240
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -449,13 +451,9 @@ void FlowController::initializeC2() {
 
   if (!c2_initialized_) {
     configuration_->setAgentIdentifier(identifier_str);
-    state::StateManager::initialize();
-    std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+    c2_agent_ = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
 
 Review comment:
   This was and still is a `shared_ptr` cycle between `C2Agent` and `FlowController`, which is not a bad practice and the reason for many of our memory leaks.
   Since we are refactoring this we should solve this as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r398046871
 
 

 ##########
 File path: nanofi/include/cxx/C2CallbackAgent.h
 ##########
 @@ -70,6 +69,7 @@ class C2CallbackAgent : public c2::C2Agent {
     c2_ag_stop_callback *stop;
 
  private:
+    utils::ThreadPool<utils::TaskRescheduleInfo> thread_pool_;
 
 Review comment:
   Oh well, the thread_pool_ shouldn't be here at all. This file requires no changes. I forgot to remove it from the previous commit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r387232703
 
 

 ##########
 File path: C2.md
 ##########
 @@ -49,6 +49,11 @@ Release 0.6.0: Please note that all c2 properties now exist as `nifi.c2.*`. If y
 files contain the former naming convention of `c2.*`, we will continue to support that as
 an alternate key, but you are encouraged to switch your configuration options as soon as possible.
 
 
 Review comment:
   This will not be true anymore. Will take care of this

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r398035081
 
 

 ##########
 File path: nanofi/include/cxx/C2CallbackAgent.h
 ##########
 @@ -70,6 +69,7 @@ class C2CallbackAgent : public c2::C2Agent {
     c2_ag_stop_callback *stop;
 
  private:
+    utils::ThreadPool<utils::TaskRescheduleInfo> thread_pool_;
 
 Review comment:
   Since this class inherits from C2Agent, it should also inherit its `ThreadPool`, so there should be no need for 2 `ThreadPool`s in `C2CallbackAgent`.
   
   Do you think it would make sense to revert C2Agent to take the thread pool as a constructor argument instead of creating it, in the spirit of the "[Tell, Don't Ask](https://www.martinfowler.com/bliki/TellDontAsk.html)" principle, the "[Single-responsibility principle](https://en.wikipedia.org/wiki/Single-responsibility_principle)" (object creation is a reponsibility) and "[Inversion of control](https://en.wikipedia.org/wiki/Inversion_of_control)"? This approach would solve the issue by making it possible to reuse the same thread pool for both classes, while leaving option to use separate thread pools if needed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390477569
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -304,23 +305,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   virtual void enableAllControllerServices();
 
   /**
-   * Retrieves all root response nodes from this source.
-   * @param metric_vector -- metrics will be placed in this vector.
-   * @return result of the get operation.
-   *  0 Success
-   *  1 No error condition, but cannot obtain lock in timely manner.
-   *  -1 failure
+   * Retrieves metrics node
+   * @return metrics response node
    */
-  virtual int16_t getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass);
+  virtual std::shared_ptr<state::response::ResponseNode> getMetricsNode() const;
+
+  /**
+   * Retrieves root nodes configured to be included in heartbeat
+   * @param includeManifest -- determines if manifest is to be included
+   * @return a list of response nodes
+   */
+  virtual std::vector<std::shared_ptr<state::response::ResponseNode>> getHeartbeatNodes(bool includeManifest) const;
 
 Review comment:
   I think `root_response_nodes_` owns the objects and this function shares this ownership with the callers. Whether this shared ownership is needed or not is not clear to me.
   We can definitely not use `unique_ptr`s for the returned pointers here, as ownership belongs to `root_response_nodes_`, but we could maybe use `state::response::ResponseNode*` (i.e. observer ptr) if the lifetime of the nodes don't necessitate shared ownership.
   
   Additionally, `root_response_nodes_` is protected, which means it's part of our API and contains `shared_ptr`s, so we're not going to get `unique_ptr`s there.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392364257
 
 

 ##########
 File path: extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
 ##########
 @@ -16,132 +16,48 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
 #undef NDEBUG
-#include <cassert>
-#include <utility>
-#include <chrono>
-#include <fstream>
-#include <memory>
-#include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <iostream>
-#include <sstream>
-#include "HTTPClient.h"
-#include "InvokeHTTP.h"
 #include "TestBase.h"
-#include "utils/StringUtils.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "core/ProcessGroup.h"
-#include "core/yaml/YamlConfiguration.h"
-#include "FlowController.h"
-#include "properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "io/StreamFactory.h"
-#include "CivetServer.h"
-#include "RemoteProcessorGroupPort.h"
-#include "core/ConfigurableComponent.h"
-#include "controllers/SSLContextService.h"
-#include "TestServer.h"
 #include "c2/C2Agent.h"
-#include "protocols/RESTReceiver.h"
+#include "protocols/RESTProtocol.h"
 #include "protocols/RESTSender.h"
+#include "protocols/RESTReceiver.h"
 #include "HTTPIntegrationBase.h"
-#include "agent/build_description.h"
-#include "processors/LogAttribute.h"
+#include "HTTPHandlers.h"
 
-class Responder : public CivetHandler {
+class LightWeightC2Handler : public HeartbeatHandler {
  public:
-  explicit Responder(bool isSecure)
-      : isSecure(isSecure) {
+  explicit LightWeightC2Handler(bool isSecure)
+      : HeartbeatHandler(isSecure) {
   }
 
-  std::string readPost(struct mg_connection *conn) {
-    std::string response;
-    int blockSize = 1024 * sizeof(char), readBytes;
+  virtual ~LightWeightC2Handler() = default;
 
-    char buffer[1024];
-    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
-      response.append(buffer, 0, (readBytes / sizeof(char)));
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *)  {
+    if (calls_ == 0) {
+      verifyJsonHasAgentManifest(root);
+    } else {
+      assert(root.HasMember("agentInfo") == true);
+      assert(root["agentInfo"].HasMember("agentManifest") == false);
 
 Review comment:
   redundant `== true`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392274155
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -172,6 +173,9 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
 
 FlowController::~FlowController() {
   stop(true);
+  if (c2_agent_)
+    c2_agent_->stop();
+  thread_pool_.shutdown();
 
 Review comment:
   Why was `thread_pool_.shutdown()` moved here? Isn't it unsafe to shut down the thread pool before stopping the schedulers?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r401721628
 
 

 ##########
 File path: libminifi/src/utils/BackTrace.cpp
 ##########
 @@ -80,25 +80,21 @@ BackTrace TraceResolver::getBackTrace(std::string thread_name, std::thread::nati
   // lock so that we only perform one backtrace at a time.
 #ifdef HAS_EXECINFO
   std::lock_guard<std::mutex> lock(mutex_);
-
-  caller_handle_ = pthread_self();
-  thread_handle_ = thread_handle;
   trace_ = BackTrace(std::move(thread_name));
 
-  if (0 == thread_handle_ || pthread_equal(caller_handle_, thread_handle)) {
+  if (0 == thread_handle || pthread_equal(pthread_self(), thread_handle)) {
     pull_trace();
   } else {
-    if (thread_handle_ == 0) {
+    if (thread_handle == 0) {
       return std::move(trace_);
     }
     emplace_handler();
-    if (pthread_kill(thread_handle_, SIGUSR2) != 0) {
+    std::unique_lock<std::mutex> ulock(trace_mutex_);
+    if (pthread_kill(thread_handle, SIGUSR2) != 0) {
       return std::move(trace_);
     }
-    sigset_t mask;
-    sigfillset(&mask);
-    sigdelset(&mask, SIGUSR2);
-    sigsuspend(&mask);
 
 Review comment:
   There was a race condition here which lead to a deadlock.
   Caller thread (Thread that is interested in pulling backtrace of other thread) acquired lock on a mutex (worker_queue_mutex_), and sent a SIGUSR2 signal to another thread (thread_handle_) but before the caller thread goes into a suspended state or may be even before it removes the SIGUSR2 from its blocking signal set (sigdelset) , the callee while executing its thread handler, sent a SIGUSR2 signal back to the caller and at this time the caller thread will run through its signal handler and after returning from signal handler it goes back to suspended state. From this state there is no way of waking it up.
   The callee was waiting on a condition variable tied to the mutex caller was holding thus creating a deadlock.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390460122
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    (void)conn;
 
 Review comment:
   You can avoid unused warnings by not naming the argument variable. In this case the name doesn't provide extra information (for readers) over the type.
   
   http://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#Rf-unused

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393058135
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPIntegrationBase.h
 ##########
 @@ -91,4 +87,55 @@ void CoapIntegrationBase::setUrl(std::string url, CivetHandler *handler) {
   }
 }
 
+class VerifyC2Base : public CoapIntegrationBase {
+ public:
+  explicit VerifyC2Base(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  virtual void testSetup() {
+    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setDebug<LogTestController>();
+  }
+
+  virtual void queryRootProcessGroup(std::shared_ptr<core::ProcessGroup> pg) {
 
 Review comment:
   unused parameter 'pg'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390665167
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
+    (void)conn;
 
 Review comment:
   Good point. Will remove the variable name from the argument list.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r391670955
 
 

 ##########
 File path: nanofi/include/cxx/C2CallbackAgent.h
 ##########
 @@ -47,7 +47,10 @@ class C2CallbackAgent : public c2::C2Agent {
 
  public:
 
-  explicit C2CallbackAgent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<state::StateMonitor> &updateSink, const std::shared_ptr<Configure> &configure);
+  explicit C2CallbackAgent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller,
 
 Review comment:
   This class is not used anywhere in the code today. It will be removed later when nanofi c2 work gets completed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393057215
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *) {
+    verifyJsonHasAgentManifest(root);
+  }
+
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
+  }
+
+  void verify(struct mg_connection *conn) {
+    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());
 
 Review comment:
   variable 'ok' set but not used

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389970518
 
 

 ##########
 File path: libminifi/include/core/state/nodes/MetricsBase.h
 ##########
 @@ -208,26 +208,28 @@ class NodeReporter {
   virtual ~NodeReporter() {
   }
 
+  //This function will be removed when we remove TreeUpdateListener class
 
 Review comment:
   👍 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r387232949
 
 

 ##########
 File path: conf/minifi.properties
 ##########
 @@ -52,6 +52,8 @@ nifi.database.content.repository.directory.default=${MINIFI_HOME}/content_reposi
 #nifi.c2.rest.url=
 #nifi.c2.rest.url.ack=
 nifi.c2.root.classes=DeviceInfoNode,AgentInformation,FlowInformation
+## Minimize heartbeat payload size by excluding agent manifest from the heartbeat
 
 Review comment:
   This will not be true anymore

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r391740598
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -173,9 +173,9 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
 
 FlowController::~FlowController() {
   stop(true);
+  c2_agent_->stop();
 
 Review comment:
   Are you sure we can  avoid the nullcheck here?
   I don't think we have C2 agent in case c2 is not configured in the minifi.properties file

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r406730077
 
 

 ##########
 File path: libminifi/include/utils/BackTrace.h
 ##########
 @@ -81,13 +83,32 @@ void pull_trace(uint8_t frames_to_skip = 1);
  */
 void emplace_handler();
 
+class Lock {
 
 Review comment:
   Why do we need this? `std::unique_lock` would be the standard choice here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r393056992
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,103 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, sizeof(buffer))) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"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());
+  }
+
+  void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn) {
+    std::string heartbeat_response = "{\"operation\" : \"heartbeat\",\"requested_operations\": [  {"
+          "\"operation\" : \"" + operation + "\","
+          "\"operationid\" : \"" + operationId + "\","
+          "\"operand\": \"" + operand + "\"}]}";
+
+      mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                heartbeat_response.length());
+      mg_printf(conn, "%s", heartbeat_response.c_str());
+  }
+
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+    bool found = false;
+    assert(root.HasMember("agentInfo") == true);
+    assert(root["agentInfo"].HasMember("agentManifest") == 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);
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *) {
+    verifyJsonHasAgentManifest(root);
+  }
+
+  virtual void handleAcknowledge(const rapidjson::Document& root) {
 
 Review comment:
   unused parameter 'root'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r390664802
 
 

 ##########
 File path: extensions/http-curl/tests/HTTPHandlers.h
 ##########
 @@ -343,4 +345,104 @@ class DeleteTransactionResponder : public CivetHandler {
   std::string response_code;
 };
 
+class HeartbeatHandler : public CivetHandler {
+ public:
+  explicit HeartbeatHandler(bool isSecure)
+      : isSecure(isSecure) {
+  }
+
+  std::string readPost(struct mg_connection *conn) {
+    std::string response;
+    int blockSize = 1024 * sizeof(char), readBytes;
+
+    char buffer[1024];
+    while ((readBytes = mg_read(conn, buffer, blockSize)) > 0) {
+      response.append(buffer, 0, (readBytes / sizeof(char)));
+    }
+    return response;
+  }
+
+  void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"FlowController\"  } ]}";
 
 Review comment:
   I like the neatness of using string literals however, I would rather build a json with some json library and serialize it. But I would keep it as it is for the test code. If we ever want to reuse this response string elsewhere, I would be happy to refactor this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392277388
 
 

 ##########
 File path: libminifi/src/FlowController.cpp
 ##########
 @@ -907,29 +908,44 @@ int16_t FlowController::clearConnection(const std::string &connection) {
   return -1;
 }
 
-int16_t FlowController::getResponseNodes(std::vector<std::shared_ptr<state::response::ResponseNode>> &metric_vector, uint16_t metricsClass) {
+std::shared_ptr<state::response::ResponseNode> FlowController::getMetricsNode(const std::string& metricsClass) const {
   std::lock_guard<std::mutex> lock(metrics_mutex_);
-
-  for (auto metric : root_response_nodes_) {
-    metric_vector.push_back(metric.second);
+  if (!metricsClass.empty()) {
+    const auto citer = component_metrics_.find(metricsClass);
+    if (citer != component_metrics_.end()) {
+      return citer->second;
+    }
+  } else {
+    const auto iter = root_response_nodes_.find("metrics");
+    if (iter != root_response_nodes_.end()) {
+      return iter->second;
+    }
 
 Review comment:
   It's possible to avoid code duplication by saying:
   `const auto metrics_key = !metricsClass.empty() ? metricsClass : std::string{"metrics"};`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r396398966
 
 

 ##########
 File path: libminifi/src/c2/C2Agent.cpp
 ##########
 @@ -32,24 +32,34 @@
 #include "utils/file/FileUtils.h"
 #include "utils/file/FileManager.h"
 #include "utils/HTTPClient.h"
+#include "utils/GeneralUtils.h"
+#include "utils/Monitors.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace c2 {
 
-C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<state::StateMonitor> &updateSink,
-                 const std::shared_ptr<Configure> &configuration)
+std::shared_ptr<utils::IdGenerator> C2Agent::id_generator_ = utils::IdGenerator::getIdGenerator();
 
 Review comment:
   I don't think this contributes to the current issue, but as we discussed previously, `utils::IdGenerator::getIdGenerator()` should be used whenever the `IdGenerator` singleton instance is required, and not assigned and stored to a global static variable.
   
   The whole point of `utils::IdGenerator::getIdGenerator()` is to make it a function static initialization, because we generally want to avoid global static initializations, as they can be performed at dynamic library load times, which, especially on Windows, significantly restricts the type of OS APIs we can use and can cause serious, hard-to-detect issues.
   
   This reduces the IdGenerator to global static initialization. Please refactor C2Agent to call the `utils::IdGenerator::getIdGenerator()` singleton getter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389967670
 
 

 ##########
 File path: libminifi/include/c2/C2Agent.h
 ##########
 @@ -171,13 +159,20 @@ class C2Agent : public state::UpdateController, public state::response::Response
    */
   bool update_property(const std::string &property_name, const std::string &property_value,  bool persist = false);
 
+  /**
+   * Creates configuration options C2 payload for response
+   */
+  C2Payload prepareConfigurationOptions(const C2ContentResponse &resp) const;
+
+  void execute(const std::function<state::Update()>& function);
+
   std::timed_mutex metrics_mutex_;
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> metrics_map_;
 
   /**
-     * Device information stored in the metrics format
-     */
-    std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
+   * Device information stored in the metrics format
+   */
+  std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
 
 Review comment:
   Wonder if we could store unique ptrs in this as well? I know this is old code, so in case it has a bigger impact, I'm ok with a follow-up as well. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r392936747
 
 

 ##########
 File path: extensions/http-curl/tests/C2DescribeManifestTest.cpp
 ##########
 @@ -0,0 +1,66 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include <string>
+#include "TestBase.h"
+#include "InvokeHTTP.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+
+class DescribeManifestHandler: public HeartbeatHandler {
+public:
+
+  explicit DescribeManifestHandler(bool isSecure)
+      : HeartbeatHandler(isSecure) {
+  }
+
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection * conn) {
 
 Review comment:
   unused parameter 'root'

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
msharee9 commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r401721628
 
 

 ##########
 File path: libminifi/src/utils/BackTrace.cpp
 ##########
 @@ -80,25 +80,21 @@ BackTrace TraceResolver::getBackTrace(std::string thread_name, std::thread::nati
   // lock so that we only perform one backtrace at a time.
 #ifdef HAS_EXECINFO
   std::lock_guard<std::mutex> lock(mutex_);
-
-  caller_handle_ = pthread_self();
-  thread_handle_ = thread_handle;
   trace_ = BackTrace(std::move(thread_name));
 
-  if (0 == thread_handle_ || pthread_equal(caller_handle_, thread_handle)) {
+  if (0 == thread_handle || pthread_equal(pthread_self(), thread_handle)) {
     pull_trace();
   } else {
-    if (thread_handle_ == 0) {
+    if (thread_handle == 0) {
       return std::move(trace_);
     }
     emplace_handler();
-    if (pthread_kill(thread_handle_, SIGUSR2) != 0) {
+    std::unique_lock<std::mutex> ulock(trace_mutex_);
+    if (pthread_kill(thread_handle, SIGUSR2) != 0) {
       return std::move(trace_);
     }
-    sigset_t mask;
-    sigfillset(&mask);
-    sigdelset(&mask, SIGUSR2);
-    sigsuspend(&mask);
 
 Review comment:
   There was a race condition here which lead to a deadlock.
   Caller thread (Thread that is interested in pulling backtrace of other thread) acquired lock on a mutex (worker_queue_mutex_), that is outside this function and sent a SIGUSR2 signal to another thread (thread_handle_) but before the caller thread goes into a suspended state or may be even before it removes the SIGUSR2 from its blocking signal set (sigdelset) , the callee while executing its thread handler, sent a SIGUSR2 signal back to the caller and at this time the caller thread will run through its signal handler and after returning from signal handler it goes back to suspended state. From this state there is no way of waking it up.
   The callee was waiting on a condition variable tied to the mutex caller was holding thus creating a deadlock.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #743: Minificpp 1169 - Simplify C2 metrics collection and reporting
URL: https://github.com/apache/nifi-minifi-cpp/pull/743#discussion_r389966149
 
 

 ##########
 File path: libminifi/include/FlowController.h
 ##########
 @@ -406,22 +409,27 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
 
   std::chrono::steady_clock::time_point start_time_;
 
-  std::mutex metrics_mutex_;
+  mutable std::mutex metrics_mutex_;
   // root_nodes cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
+
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> device_information_;
 
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> component_metrics_;
 
   std::map<uint8_t, std::vector<std::shared_ptr<state::response::ResponseNode>>> component_metrics_by_id_;
+
   // metrics last run
   std::chrono::steady_clock::time_point last_metrics_capture_;
 
  private:
   std::shared_ptr<logging::Logger> logger_;
   std::string serial_number_;
+
+  std::shared_ptr<state::UpdateController> c2_agent_;
 
 Review comment:
   As far as I see the code, the c2 agent is created within this class and no reference (shared pointer instance) is passed anywhere, so this could be a simple unique ptr to indicate that flowcontroller owns the agent.
   Feel free to fix me if I missed something in this aspect. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services