You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by phrocker <gi...@git.apache.org> on 2017/09/05 17:07:37 UTC

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

GitHub user phrocker opened a pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134

    MINIFI-339: Add C2 base allowing for 1 protocol and n heartbeat repor…

    …ters
    
    MINIFI-339: Add GetTCP Processor
    MINIFI-339: Add listener server
    MINIFI-339: Update to listener
    MINIFI-339: Resolve Issue with stack based processor nodes losing scope
    
    MINIFI-369: Update ListenHTTP processor to allow transfer encoding
    
    MINIFI-339: Update rest receiver instantiation and fix issue found in GetFile
    
    MINIFI-339: Rename content to operational arguments
    
    MINIFI-371: remove virtual destructors when not needed
    
    MINIFI-339: Fixing issues with GetTCP
    
    MINIFI-378: Resolve issues with shutdown. Took the approach to call notifyStop at the destructor to avoid larger changes
    
    MININFI-339: fix broken test
    
    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 MINIFI-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.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/phrocker/nifi-minifi-cpp MINIFI-339

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi-minifi-cpp/pull/134.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #134
    
----
commit 3ab73825efb19e8e5c3a2b8163971e8c0db4b192
Author: Marc Parisi <ph...@apache.org>
Date:   2017-06-21T14:47:35Z

    MINIFI-339: Add C2 base allowing for 1 protocol and n heartbeat reporters
    MINIFI-339: Add GetTCP Processor
    MINIFI-339: Add listener server
    MINIFI-339: Update to listener
    MINIFI-339: Resolve Issue with stack based processor nodes losing scope
    
    MINIFI-369: Update ListenHTTP processor to allow transfer encoding
    
    MINIFI-339: Update rest receiver instantiation and fix issue found in GetFile
    
    MINIFI-339: Rename content to operational arguments
    
    MINIFI-371: remove virtual destructors when not needed
    
    MINIFI-339: Fixing issues with GetTCP
    
    MINIFI-378: Resolve issues with shutdown. Took the approach to call notifyStop at the destructor to avoid larger changes
    
    MININFI-339: fix broken test

----


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137881667
  
    --- Diff: libminifi/src/RemoteProcessorGroupPort.cpp ---
    @@ -221,51 +223,28 @@ void RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo() {
     
       if (!rest_user_name_.empty()) {
         std::string loginUrl = this->protocol_ + this->host_ + ":" + std::to_string(this->port_) + "/nifi-api/access/token";
    -    token = utils::get_token(loginUrl, this->rest_user_name_, this->rest_password_, this->securityConfig_);
    -    logger_->log_debug("Token from NiFi REST Api endpoint %s", token);
    +    utils::HTTPClient client(loginUrl, ssl_service);
    +    client.setVerbose();
    +    token = utils::get_token(client, this->rest_user_name_, this->rest_password_);
    +    logger_->log_debug("Token from NiFi REST Api endpoint %s,  %s", loginUrl, token);
         if (token.empty())
    -        return;
    +      return;
       }
     
    -  CURL *http_session = curl_easy_init();
    +  utils::HTTPClient client(fullUrl.c_str(), ssl_service);
     
    -  if (fullUrl.find("https") != std::string::npos) {
    -    this->securityConfig_.configureSecureConnection(http_session);
    -  }
    +  client.initialize("GET");
     
       struct curl_slist *list = NULL;
       if (!token.empty()) {
         std::string header = "Authorization: " + token;
         list = curl_slist_append(list, header.c_str());
    -    curl_easy_setopt(http_session, CURLOPT_HTTPHEADER, list);
    +    client.setHeaders(list);
       }
     
    -  curl_easy_setopt(http_session, CURLOPT_URL, fullUrl.c_str());
    -
    -  utils::HTTPRequestResponse content;
    -  curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION,
    -      &utils::HTTPRequestResponse::recieve_write);
    -
    -  curl_easy_setopt(http_session, CURLOPT_WRITEDATA,
    -      static_cast<void*>(&content));
    -
    -  CURLcode res = curl_easy_perform(http_session);
    -  if (list)
    -    curl_slist_free_all(list);
    -
    -  if (res == CURLE_OK) {
    -    std::string response_body(content.data.begin(), content.data.end());
    -    int64_t http_code = 0;
    -    curl_easy_getinfo(http_session, CURLINFO_RESPONSE_CODE, &http_code);
    -    char *content_type;
    -    /* ask for the content-type */
    -    curl_easy_getinfo(http_session, CURLINFO_CONTENT_TYPE, &content_type);
    -
    -    bool isSuccess = ((int32_t) (http_code / 100)) == 2
    -        && res != CURLE_ABORTED_BY_CALLBACK;
    -    bool body_empty = IsNullOrEmpty(content.data);
    -
    -    if (isSuccess && !body_empty) {
    +  if (client.submit() == CURLE_OK && client.getResponseCode() == 200) {
    --- End diff --
    
    Our curl abstraction is leaking here (referencing curl constants).


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @phrocker That sounds reasonable re: documentation. I would put some docs embedded in the code that point out what you just said (that this is using the Garcon protocol, with a link/reference to those docs). I don't think this needs to be in the end-user docs as they probably don't care about the low level protocol.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @benqiu2016 , much like the response to @achristianson , were' adhering to something that isn't published yet, so I'll pose the same question to you. 
    
     "I think we should add documentation to the c2 server or Apache wiki ( or both ? ) and link to it from here. In the interest of getting something here, which was a point discussed yesterday, I'll add an item on the apache docs and reference them from here. I plan to follow that up with an E-mail the dev list asking for input from the larger group as a whole. The same will follow suit for other protocol implementations. Does that make sense to you or do you feel we should put that documentation here? Thanks!"


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137881468
  
    --- Diff: libminifi/src/RemoteProcessorGroupPort.cpp ---
    @@ -202,14 +205,13 @@ void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context, core::Pr
         context->yield();
         session->rollback();
       }
    -
    -
    -  throw std::exception();
    +  logger_->log_info("on trigger throw exception");
    +  return;
     }
     
     void RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo() {
       if (this->host_.empty() || this->port_ == -1 || this->protocol_.empty())
    --- End diff --
    
    Missing curlies.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137865638
  
    --- Diff: libminifi/include/core/state/UpdateController.h ---
    @@ -0,0 +1,255 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +#define LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +
    +#include <string>
    +#include "utils/ThreadPool.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +
    +enum class UpdateState {
    +  INITIATE,
    +  FULLY_APPLIED,
    +  READ_COMPLETE,
    +  PARTIALLY_APPLIED,
    +  NOT_APPLIED,
    +  SET_ERROR,
    +  READ_ERROR,
    +  NESTED  // multiple updates embedded into one
    +
    +};
    +
    +/**
    + * Represents the status of an update operation.
    + *
    + */
    +class UpdateStatus {
    + public:
    +  UpdateStatus(UpdateState state, int16_t reason = 0);
    +
    +  UpdateStatus(const UpdateStatus &other);
    +
    +  UpdateStatus(const UpdateStatus &&other);
    +
    +  UpdateState getState() const;
    +
    +  std::string getError() const;
    +
    +  int16_t getReadonCode() const;
    +
    +  UpdateStatus &operator=(const UpdateStatus &&other);
    +
    +  UpdateStatus &operator=(const UpdateStatus &other);
    +   private:
    --- End diff --
    
    Indentation


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137570255
  
    --- Diff: libminifi/include/core/state/StateManager.h ---
    @@ -0,0 +1,127 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_CONTROLLABLE_H_
    +#define LIBMINIFI_INCLUDE_C2_CONTROLLABLE_H_
    --- End diff --
    
    The include guard here appears to be inconsistent with the file name.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    One more thing that struck me during testing: it seems broken that we attempt to periodically send heartbeats when c2 has not been configured. Would it be possible to not attempt heartbeats if a c2 server is not set, or otherwise add a config flag to enable/disable c2?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by benqiu2016 <gi...@git.apache.org>.
Github user benqiu2016 commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137858437
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    +
    +  /**
    +   * Returns raw data.
    +   */
    +  std::string getRawData() const;
    +
    +  /**
    +   * Add a nested payload.
    +   * @param payload payload to move into this object.
    +   */
    +  void addPayload(const C2Payload &&payload);
    +  /**
    +   * Get nested payloads.
    +   */
    +  const std::vector<C2Payload> &getNestedPayloads() const;
    +
    +  C2Payload &operator=(const C2Payload &&other);
    +  C2Payload &operator=(const C2Payload &other);
    +
    + protected:
    +
    +  // identifier for this payload.
    +  std::string ident_;
    +
    +  std::string label_;
    +
    +  std::vector<C2Payload> payloads_;
    +
    +  std::vector<C2ContentResponse> content_;
    +
    +  Operation op_;
    +
    +  bool raw_;
    +
    +  std::string raw_data_;
    +
    +  bool isResponse;
    +
    +};
    +
    +} /* namesapce c2 */
    +} /* namespace minifi */
    +} /* namespace nifi */
    +} /* namespace apache */
    +} /* namespace org */
    +
    +#endif /* LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_ */
    --- End diff --
    
    for payload, have we consider using protobuf or avro format for the same?



---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137879427
  
    --- Diff: libminifi/include/processors/InvokeHTTP.h ---
    @@ -101,9 +105,9 @@ class InvokeHTTP : public core::Processor {
       static core::Relationship RelNoRetry;
       static core::Relationship RelFailure;
     
    -  void onTrigger(core::ProcessContext *context, core::ProcessSession *session);
    -  void initialize();
    -  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory);
    +  virtual void onTrigger(core::ProcessContext *context, core::ProcessSession *session);
    +  virtual void initialize();
    +  virtual void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory);
    --- End diff --
    
    Probably want to use override instead of virtual.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @achristianson It slipped my mind but was hoping to get ideas on what to do when we have an invalid c2 configuration. C2 is a local element too, so at some point we'll have scripts that leverage this capability to perform local operations, so I imagine we'll always have C2 running ( which was the plan ) but the in the event that a protocol isn't properly configured i.e. no URL we should disable that protocol entirely. thoughts? 
    
    Disabling the command and control element entirely seems antithetical to command and control especially since we want local capabilities, but not trying the protocol if it didn't fully initialize is advantageous and something I completely forgot. Thoughts on that?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137557066
  
    --- Diff: CMakeLists.txt ---
    @@ -35,6 +35,8 @@ ENDIF(POLICY CMP0048)
     include(CheckCXXCompilerFlag)
     CHECK_CXX_COMPILER_FLAG("-std=c++11 " COMPILER_SUPPORTS_CXX11)
     CHECK_CXX_COMPILER_FLAG("-std=c++0x " COMPILER_SUPPORTS_CXX0X)
    +SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ")
    +SET(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}")
    --- End diff --
    
    What are these for?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r139702066
  
    --- Diff: libminifi/include/core/Processor.h ---
    @@ -153,7 +154,8 @@ class Processor : public Connectable, public ConfigurableComponent, public std::
       }
       // decrement Active Task Counts
       void decrementActiveTask(void) {
    -    active_tasks_--;
    +    if (active_tasks_ > 0)
    +      active_tasks_--;
    --- End diff --
    
    Since the input ( stop command for example ) is user provided, this is only a protection. Decrement only occurs when an exception occurs during on trigger. If this happens just after unschedule occurs ( and the active tasks is set to 0 ) then we could arrive at -1. I didn't want to impose additional locking. 
    
    Not ideal, and this can be solved more elegantly by not decrementing active tasks during an exception since the number isn't currently important ( just that tasks > 0 )...but I did not want to make additional changes here. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137885309
  
    --- Diff: CMakeLists.txt ---
    @@ -35,6 +35,8 @@ ENDIF(POLICY CMP0048)
     include(CheckCXXCompilerFlag)
     CHECK_CXX_COMPILER_FLAG("-std=c++11 " COMPILER_SUPPORTS_CXX11)
     CHECK_CXX_COMPILER_FLAG("-std=c++0x " COMPILER_SUPPORTS_CXX0X)
    +SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ")
    +SET(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}")
    --- End diff --
    
    Got it. If we add them back, and I understand correctly how this all is working, then I think we should use distinct vars rather than exact copies of the cmake parameters. Something like CXX_OPTIMIZATION_FLAGS. I do think optimizations are going to be desirable. I almost feel like we need some kind of automated/reproducible way to measure things before getting too fancy with optimizations. I.e. a microbenchmark suite, something to measure bin size, and something to measure memory usage.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137841080
  
    --- Diff: libminifi/include/c2/C2Agent.h ---
    @@ -0,0 +1,203 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2AGENT_H_
    +#define LIBMINIFI_INCLUDE_C2_C2AGENT_H_
    +
    +#include <utility>
    +#include <functional>
    +#include <future>
    +#include <memory>
    +#include <mutex>
    +#include <thread>
    +#include "core/state/UpdateController.h"
    +#include "core/state/metrics/MetricsBase.h"
    +#include "C2Payload.h"
    +#include "C2Protocol.h"
    +#include "io/validation.h"
    +#include "protocols/Protocols.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +/**
    + * Purpose and Justification: C2 agent will be the mechanism that will abstract the protocol to do the work.
    + *
    + * The protocol represents a transformation layer into the objects seen in C2Payload. That transformation may
    + * be minimal or extreme, depending on the protocol itself.
    + *
    + * Metrics Classes defined here:
    + *
    + *   0 HeartBeat --  RESERVED
    + *   1-255 Defined by the configuration file.
    + */
    +class C2Agent : public state::UpdateController, public state::metrics::MetricsSink, public std::enable_shared_from_this<C2Agent> {
    + public:
    +
    +  C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller,
    +          const std::shared_ptr<state::StateMonitor> &updateSink,
    +          const std::shared_ptr<Configure> &configure);
    +
    +  virtual ~C2Agent() {
    +
    +  }
    +
    +  /**
    +   * Sends the heartbeat to ths server. Will include metrics
    +   * in the payload if they exist.
    +   */
    +  void performHeartBeat();
    +
    +  virtual std::vector<std::function<state::Update()>> getFunctions() {
    +    return functions_;
    +  }
    +
    +  /**
    +   * Sets the metric within this sink
    +   * @param metric metric to set
    +   * @param return 0 on success, -1 on failure.
    +   */
    +  virtual int16_t setMetrics(const std::shared_ptr<state::metrics::Metrics> &metric);
    +
    + protected:
    +
    +  /**
    +   * Configure the C2 agent
    +   */
    +  void configure(const std::shared_ptr<Configure> &configure, bool reconfigure = true);
    +
    +  /**
    +   * Serializes metrics into a payload.
    +   * @parem parent_paylaod parent payload into which we insert the newly generated payload.
    +   * @param name name of this metric
    +   * @param metrics metrics to include.
    +   */
    +  void serializeMetrics(C2Payload &parent_payload, const std::string &name, const std::vector<state::metrics::MetricResponse> &metrics);
    +
    +  /**
    +   * Extract the payload
    +   * @param resp payload to be moved into the function.
    +   */
    +  void extractPayload(const C2Payload &&resp);
    +
    +  /**
    +   * Extract the payload
    +   * @param payload reference.
    +   */
    +  void extractPayload(const C2Payload &resp);
    +
    +  /**
    +   * Enqueues a C2 server response for us to evaluate and parse.
    +   */
    +  void enqueue_c2_server_response(C2Payload &&resp) {
    +    std::lock_guard<std::timed_mutex> lock(queue_mutex);
    +    responses.push_back(std::move(resp));
    --- End diff --
    
    Wondering if we couldn't use the concurrent queue here.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137817535
  
    --- Diff: libminifi/include/FlowController.h ---
    @@ -158,7 +176,7 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
       // first it will validate the payload with the current root node config for flowController
       // like FlowController id/name is the same and new version is greater than the current version
       // after that, it will apply the configuration
    -  bool applyConfiguration(std::string &configurePayload);
    +  bool applyConfiguration(const std::string &configurePayload);
    --- End diff --
    
    I don't think 339 introduced this, but we are very tightly coupled with YAML (and strings) here. FlowController should ideally be agnostic to configuration formats and take in/apply some kind of abstract config structure.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @achristianson MiNiFi adheres to a protocol defined by Garcon. I think we should add documentation to the c2 server or Apache wiki ( or both ? ) and link to it from here. In the interest of getting something here, which was a point discussed yesterday, I'll add an item on the apache docs and reference them from here. I plan to follow that up with an E-mail the dev list asking for input from the larger group as a whole. The same will follow suit for other protocol implementations.  Does that make sense to you or do you feel we should put that documentation here? Thanks!


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    Note that Travis is failing/segfaulting on a test. We're green on travis right now, so would hate to see it turn back red.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137866970
  
    --- Diff: libminifi/include/core/state/metrics/MetricsBase.h ---
    @@ -0,0 +1,167 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    +#define LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    --- End diff --
    
    Inconsistent include guard?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r139696520
  
    --- Diff: libminifi/include/FlowController.h ---
    @@ -158,7 +176,7 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
       // first it will validate the payload with the current root node config for flowController
       // like FlowController id/name is the same and new version is greater than the current version
       // after that, it will apply the configuration
    -  bool applyConfiguration(std::string &configurePayload);
    +  bool applyConfiguration(const std::string &configurePayload);
    --- End diff --
    
    FlowConfiguratin should be sanitized of YAML related variable names. Unfortunately this was reintroduced in a recent PR. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137587733
  
    --- Diff: libminifi/include/core/state/metrics/SystemMetrics.h ---
    @@ -0,0 +1,111 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_CORE_STATE_METRICS_SYSMETRICS_H_
    +#define LIBMINIFI_INCLUDE_CORE_STATE_METRICS_SYSMETRICS_H_
    +
    +#include "core/Resource.h"
    +#include <sstream>
    +#include <map>
    +#ifndef _WIN32
    +#include <sys/utsname.h>
    +#endif
    +#include "MetricsBase.h"
    +#include "Connection.h"
    +#include "DeviceInformation.h"
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +namespace metrics {
    +
    +/**
    + * Justification and Purpose: Provides Connection queue metrics. Provides critical information to the
    + * C2 server.
    --- End diff --
    
    Docs here seem to be mismatched with the class. This class is for system metrics, not queue metrics.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137877541
  
    --- Diff: libminifi/include/core/state/metrics/MetricsBase.h ---
    @@ -0,0 +1,167 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    +#define LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    +
    +#include <vector>
    +#include <memory>
    +#include <string>
    +#include "core/Core.h"
    +#include "core/Connectable.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +namespace metrics {
    +
    +struct MetricResponse {
    +  std::string name;
    +  std::string value;
    +  std::vector<MetricResponse> children;
    +  MetricResponse &operator=(const MetricResponse &other) {
    +    name = other.name;
    +    value = other.value;
    +    children = other.children;
    +    return *this;
    +  }
    +};
    +
    +/**
    + * Purpose: Defines a metric. serialization is intended to be thread safe.
    + */
    +class Metrics : public core::Connectable
    +{
    + public:
    +  Metrics()
    +      : core::Connectable("metric", 0) {
    +  }
    +
    +  Metrics(std::string name, uuid_t uuid)
    +      : core::Connectable(name, uuid)
    +  {
    +  }
    +  virtual ~Metrics() {
    +
    +  }
    +  virtual std::string getName() = 0;
    +
    +  virtual std::vector<MetricResponse> serialize() = 0;
    +
    +  virtual void yield() {
    +  }
    +  virtual bool isRunning() {
    +    return true;
    +  }
    +  virtual bool isWorkAvailable() {
    +    return true;
    +  }
    +
    +};
    +
    +/**
    + * Purpose: Defines a metric that
    + */
    +class DeviceMetric : public Metrics {
    + public:
    +  DeviceMetric(std::string name, uuid_t uuid)
    +      : Metrics(name, uuid)
    +  {
    +  }
    +};
    +
    +/**
    + * Purpose: Retrieves Metrics from the defined class. The current Metric, which is a consumable for any reader of Metrics must have the ability to set metrics.
    + *
    + */
    +class MetricsSource
    +{
    + public:
    +
    +  MetricsSource() {
    +
    +  }
    +
    +  virtual ~MetricsSource() {
    +  }
    +
    +  /**
    +   * Retrieves all metrics 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
    +   */
    +  virtual int16_t getMetrics(std::vector<std::shared_ptr<Metrics>> &metric_vector) = 0;
    +
    +};
    +
    +class MetricsReporter
    +{
    + public:
    +
    +  MetricsReporter() {
    +
    +  }
    +
    +  virtual ~MetricsReporter() {
    +  }
    +
    +  /**
    +   * Retrieves all emtrics 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
    --- End diff --
    
    IMO we should have either an enum response type or at least error constants for cases where we want to avoid using exceptions.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137862598
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    +
    +  /**
    +   * Returns raw data.
    +   */
    +  std::string getRawData() const;
    +
    +  /**
    +   * Add a nested payload.
    +   * @param payload payload to move into this object.
    +   */
    +  void addPayload(const C2Payload &&payload);
    +  /**
    +   * Get nested payloads.
    +   */
    +  const std::vector<C2Payload> &getNestedPayloads() const;
    +
    +  C2Payload &operator=(const C2Payload &&other);
    +  C2Payload &operator=(const C2Payload &other);
    +
    + protected:
    +
    +  // identifier for this payload.
    +  std::string ident_;
    +
    +  std::string label_;
    +
    +  std::vector<C2Payload> payloads_;
    +
    +  std::vector<C2ContentResponse> content_;
    +
    +  Operation op_;
    +
    +  bool raw_;
    +
    +  std::string raw_data_;
    +
    +  bool isResponse;
    +
    +};
    +
    +} /* namesapce c2 */
    +} /* namespace minifi */
    +} /* namespace nifi */
    +} /* namespace apache */
    +} /* namespace org */
    +
    +#endif /* LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_ */
    --- End diff --
    
    The intent is that any versioning is left up to the protocol. The current protocol doesn't support that, but perhaps it should. I will attempt to document this better. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137880036
  
    --- Diff: libminifi/include/utils/HTTPClient.h ---
    @@ -0,0 +1,270 @@
    +/**
    + * HTTPUtils class declaration
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef __HTTP_UTILS_H__
    +#define __HTTP_UTILS_H__
    +
    +#include <curl/curl.h>
    +#include <vector>
    +#include <iostream>
    +#include <string>
    +#include <curl/easy.h>
    +#include <uuid/uuid.h>
    +#include <regex.h>
    +#include <vector>
    +#include "controllers/SSLContextService.h"
    +#include "ByteInputCallBack.h"
    +#include "core/logging/Logger.h"
    +#include "core/logging/LoggerConfiguration.h"
    +#include "properties/Configure.h"
    +#include "io/validation.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace utils {
    +
    +struct HTTPUploadCallback {
    +  ByteInputCallBack *ptr;
    +  size_t pos;
    +};
    +
    +/**
    + * HTTP Response object
    + */
    +struct HTTPRequestResponse {
    +  std::vector<char> data;
    +
    +  /**
    +   * Receive HTTP Response.
    +   */
    +  static size_t recieve_write(char * data, size_t size, size_t nmemb,
    +                              void * p) {
    +    return static_cast<HTTPRequestResponse*>(p)->write_content(data, size,
    +                                                               nmemb);
    +  }
    +
    +  /**
    +   * Callback for post, put, and patch operations
    +   * @param buffer
    +   * @param size size of buffer
    +   * @param nitems items to add
    +   * @param insteam input stream object.
    +   */
    +
    +  static size_t send_write(char * data, size_t size, size_t nmemb, void * p) {
    +    if (p != 0) {
    +      HTTPUploadCallback *callback = (HTTPUploadCallback*) p;
    +      if (callback->pos <= callback->ptr->getBufferSize()) {
    +        char *ptr = callback->ptr->getBuffer();
    +        int len = callback->ptr->getBufferSize() - callback->pos;
    +        if (len <= 0) {
    +          return 0;
    +        }
    +        if (len > size * nmemb)
    +          len = size * nmemb;
    +        memcpy(data, callback->ptr->getBuffer() + callback->pos, len);
    +        callback->pos += len;
    +        return len;
    +      }
    +    } else {
    +      return CURL_READFUNC_ABORT;
    +    }
    +
    +    return 0;
    +  }
    +
    +  size_t write_content(char* ptr, size_t size, size_t nmemb) {
    +    data.insert(data.end(), ptr, ptr + size * nmemb);
    +    return size * nmemb;
    +  }
    +
    +};
    +
    +static void parse_url(std::string &url, std::string &host, int &port, std::string &protocol) {
    +
    --- End diff --
    
    Debating whether or not it's a good idea to parse URLs with our own code here. On one hand, it reduces the need for any library calls, but on the other hand it could get complex to support everything according to spec.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137886437
  
    --- Diff: libminifi/include/core/state/metrics/MetricsBase.h ---
    @@ -0,0 +1,167 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    +#define LIBMINIFI_INCLUDE_C2_METRICS_METRICSBASE_H_
    +
    +#include <vector>
    +#include <memory>
    +#include <string>
    +#include "core/Core.h"
    +#include "core/Connectable.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +namespace metrics {
    +
    +struct MetricResponse {
    +  std::string name;
    +  std::string value;
    +  std::vector<MetricResponse> children;
    +  MetricResponse &operator=(const MetricResponse &other) {
    +    name = other.name;
    +    value = other.value;
    +    children = other.children;
    +    return *this;
    +  }
    +};
    +
    +/**
    + * Purpose: Defines a metric. serialization is intended to be thread safe.
    + */
    +class Metrics : public core::Connectable
    +{
    + public:
    +  Metrics()
    +      : core::Connectable("metric", 0) {
    +  }
    +
    +  Metrics(std::string name, uuid_t uuid)
    +      : core::Connectable(name, uuid)
    +  {
    +  }
    +  virtual ~Metrics() {
    +
    +  }
    +  virtual std::string getName() = 0;
    +
    +  virtual std::vector<MetricResponse> serialize() = 0;
    +
    +  virtual void yield() {
    +  }
    +  virtual bool isRunning() {
    +    return true;
    +  }
    +  virtual bool isWorkAvailable() {
    +    return true;
    +  }
    +
    +};
    +
    +/**
    + * Purpose: Defines a metric that
    + */
    +class DeviceMetric : public Metrics {
    + public:
    +  DeviceMetric(std::string name, uuid_t uuid)
    +      : Metrics(name, uuid)
    +  {
    +  }
    +};
    +
    +/**
    + * Purpose: Retrieves Metrics from the defined class. The current Metric, which is a consumable for any reader of Metrics must have the ability to set metrics.
    + *
    + */
    +class MetricsSource
    +{
    + public:
    +
    +  MetricsSource() {
    +
    +  }
    +
    +  virtual ~MetricsSource() {
    +  }
    +
    +  /**
    +   * Retrieves all metrics 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
    +   */
    +  virtual int16_t getMetrics(std::vector<std::shared_ptr<Metrics>> &metric_vector) = 0;
    +
    +};
    +
    +class MetricsReporter
    +{
    + public:
    +
    +  MetricsReporter() {
    +
    +  }
    +
    +  virtual ~MetricsReporter() {
    +  }
    +
    +  /**
    +   * Retrieves all emtrics 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
    --- End diff --
    
    E.g. E_LOCK_TIMEOUT, E_GENERAL_FAILURE, something like that (if not an enum).


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137866433
  
    --- Diff: libminifi/include/core/state/UpdateController.h ---
    @@ -0,0 +1,255 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +#define LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +
    +#include <string>
    +#include "utils/ThreadPool.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +
    +enum class UpdateState {
    +  INITIATE,
    +  FULLY_APPLIED,
    +  READ_COMPLETE,
    +  PARTIALLY_APPLIED,
    +  NOT_APPLIED,
    +  SET_ERROR,
    +  READ_ERROR,
    +  NESTED  // multiple updates embedded into one
    +
    +};
    +
    +/**
    + * Represents the status of an update operation.
    + *
    + */
    +class UpdateStatus {
    + public:
    +  UpdateStatus(UpdateState state, int16_t reason = 0);
    +
    +  UpdateStatus(const UpdateStatus &other);
    +
    +  UpdateStatus(const UpdateStatus &&other);
    +
    +  UpdateState getState() const;
    +
    +  std::string getError() const;
    +
    +  int16_t getReadonCode() const;
    +
    +  UpdateStatus &operator=(const UpdateStatus &&other);
    +
    +  UpdateStatus &operator=(const UpdateStatus &other);
    +   private:
    +  UpdateState state_;
    +  std::string error_;
    +  int16_t reason_;
    +};
    +
    +class Update
    +{
    + public:
    +
    +  Update(UpdateStatus status)
    +      : status_(status) {
    +
    +  }
    +
    +  Update(const Update &other)
    +      : status_(other.status_) {
    +
    +  }
    +
    +  Update(const Update &&other)
    +      : status_(std::move(other.status_)) {
    +
    +  }
    +
    +  virtual ~Update() {
    +
    +  }
    +
    +  virtual bool validate() {
    +    return true;
    +  }
    +
    +  const UpdateStatus &getStatus() const {
    +    return status_;
    +  }
    +
    +  Update &operator=(const Update &&other) {
    +    status_ = std::move(other.status_);
    +    return *this;
    +  }
    +
    +  Update &operator=(const Update &other) {
    +    status_ = other.status_;
    +    return *this;
    +  }
    +
    + protected:
    +  UpdateStatus status_;
    +};
    +
    +/**
    + * Justification and Purpose: Update Runner reflects the post execution functors that determine if
    + * a given function that is running within a thread pool worker needs to end.
    + *
    + * Design: Simply implements isFinished and isCancelled, which it receives by way of the AfterExecute
    + * class.
    + */
    +class UpdateRunner : public utils::AfterExecute<Update>
    +{
    + public:
    +  explicit UpdateRunner(std::atomic<bool> &running)
    +      : running_(&running) {
    +  }
    +
    +  explicit UpdateRunner(UpdateRunner && other)
    +      : running_(std::move(other.running_)) {
    +
    +  }
    +
    +  ~UpdateRunner() {
    +
    +  }
    +
    +  virtual bool isFinished(const Update &result) {
    +    if ((result.getStatus().getState() == UpdateState::FULLY_APPLIED || result.getStatus().getState() == UpdateState::READ_COMPLETE) && *running_) {
    --- End diff --
    
    As a general note, it seems we're building up quite a bit of concurrent/mutex/async code. Wondering if we can simplify a bit by using queues/workers to reduce context switching and also reduce the chances of contention.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137863123
  
    --- Diff: libminifi/include/c2/protocols/RESTSender.h ---
    @@ -0,0 +1,81 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_RESTPROTOCOL_H_
    +#define LIBMINIFI_INCLUDE_C2_RESTPROTOCOL_H_
    --- End diff --
    
    Inconsistent include guard?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137821983
  
    --- Diff: libminifi/include/RemoteProcessorGroupPort.h ---
    @@ -23,13 +23,14 @@
     #include <mutex>
     #include <memory>
     #include <stack>
    -#include "utils/HTTPUtils.h"
    +#include "utils/HTTPClient.h"
    --- End diff --
    
    +1 on the rename to HTTPClient. To further improve the structural semantics, we may want to consider moving HTTPClient from utils/ and into io/, because it is a type of input/output.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137864284
  
    --- Diff: libminifi/include/core/Processor.h ---
    @@ -212,19 +217,35 @@ class Processor : public Connectable, public ConfigurableComponent, public std::
      public:
     
       // OnTrigger method, implemented by NiFi Processor Designer
    +  virtual void onTrigger(std::shared_ptr<ProcessContext> context, std::shared_ptr<ProcessSession> session){
    +    onTrigger(context.get(),session.get());
    +  }
       virtual void onTrigger(ProcessContext *context, ProcessSession *session) = 0;
       // Initialize, overridden by NiFi Process Designer
       virtual void initialize() {
       }
       // Scheduled event hook, overridden by NiFi Process Designer
    +  virtual void onSchedule(std::shared_ptr<ProcessContext> context, std::shared_ptr<ProcessSessionFactory> sessionFactory){
    +    onSchedule(context.get(),sessionFactory.get());
    +  }
       virtual void onSchedule(ProcessContext *context, ProcessSessionFactory *sessionFactory) {
       }
     
       // Check all incoming connections for work
       bool isWorkAvailable();
     
    +  void setStreamFactory(std::shared_ptr<minifi::io::StreamFactory> stream_factory) {
    --- End diff --
    
    Why not just require a valid StreamFactory when the object is constructed, and not allow it to be dynamically changed?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137833409
  
    --- Diff: CMakeLists.txt ---
    @@ -35,6 +35,8 @@ ENDIF(POLICY CMP0048)
     include(CheckCXXCompilerFlag)
     CHECK_CXX_COMPILER_FLAG("-std=c++11 " COMPILER_SUPPORTS_CXX11)
     CHECK_CXX_COMPILER_FLAG("-std=c++0x " COMPILER_SUPPORTS_CXX0X)
    +SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ")
    +SET(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}")
    --- End diff --
    
    Looking at the branch I had some optimizations and in the conflict removed them. I'm going to think a little more about adding them back in and then make a resolution of keeping these lines with the optimizations or remove the line entirely, thanks!


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    Mac & docker builds work for me, but building on CentOS7 results in this error:
    
    ```
    [ 44%] Linking CXX executable SiteToSiteRestTest
    CMakeFiles/SiteToSiteRestTest.dir/libminifi/test/integration/SiteToSiteRestTest.cpp.o: In function `org::apache::nifi::minifi::core::repository::FlowFileRepository::initialize(std::shared_ptr<org::apache::nifi::minifi::Configure> const&)':
    SiteToSiteRestTest.cpp:(.text._ZN3org6apache4nifi6minifi4core10repository18FlowFileRepository10initializeERKSt10shared_ptrINS2_9ConfigureEE[_ZN3org6apache4nifi6minifi4core10repository18FlowFileRepository10initializeERKSt10shared_ptrINS2_9ConfigureEE]+0x3b3): undefined reference to `leveldb::DB::Open(leveldb::Options const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, leveldb::DB**)'
    libminifi/libminifi.a(SiteToSiteProvenanceReportingTask.cpp.o): In function `org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::getJsonReport(org::apache::nifi::minifi::core::ProcessContext*, org::apache::nifi::minifi::core::ProcessSession*, std::vector<std::shared_ptr<org::apache::nifi::minifi::core::SerializableComponent>, std::allocator<std::shared_ptr<org::apache::nifi::minifi::core::SerializableComponent> > >&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >&)':
    SiteToSiteProvenanceReportingTask.cpp:(.text+0x2459): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    SiteToSiteProvenanceReportingTask.cpp:(.text+0x2472): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    SiteToSiteProvenanceReportingTask.cpp:(.text+0x2ab0): undefined reference to `Json::StyledWriter::write[abi:cxx11](Json::Value const&)'
    libminifi/libminifi.a(RemoteProcessorGroupPort.cpp.o): In function `org::apache::nifi::minifi::RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo()':
    RemoteProcessorGroupPort.cpp:(.text+0x3aaf): undefined reference to `Json::Reader::parse(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, Json::Value&, bool)'
    libminifi/libminifi.a(RESTReceiver.cpp.o): In function `org::apache::nifi::minifi::c2::RESTReceiver::heartbeat(org::apache::nifi::minifi::c2::C2Payload const&)':
    RESTReceiver.cpp:(.text+0x2317): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x23c2): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x2592): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x25ac): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x25f3): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x260c): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x268c): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x279b): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTReceiver.cpp:(.text+0x2811): undefined reference to `Json::StyledWriter::write[abi:cxx11](Json::Value const&)'
    libminifi/libminifi.a(RESTSender.cpp.o): In function `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::minifi::c2::Direction, bool)':
    RESTSender.cpp:(.text+0x204e): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x20f9): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x22c9): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x22e3): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x232a): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x2343): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x23c3): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x24d2): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTSender.cpp:(.text+0x2548): undefined reference to `Json::StyledWriter::write[abi:cxx11](Json::Value const&)'
    libminifi/libminifi.a(RESTProtocol.cpp.o): In function `org::apache::nifi::minifi::c2::RESTProtocol::parseJsonResponse(org::apache::nifi::minifi::c2::C2Payload const&, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> >)':
    RESTProtocol.cpp:(.text+0x1d39): undefined reference to `Json::Reader::parse(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&, Json::Value&, bool)'
    RESTProtocol.cpp:(.text+0x1dbc): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x1e0d): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x1fc5): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x207a): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x20e9): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x2181): undefined reference to `Json::Value::getMemberNames[abi:cxx11]() const'
    RESTProtocol.cpp:(.text+0x2209): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) const'
    RESTProtocol.cpp:(.text+0x221e): undefined reference to `Json::Value::asString[abi:cxx11]() const'
    libminifi/libminifi.a(RESTProtocol.cpp.o): In function `org::apache::nifi::minifi::c2::RESTProtocol::serializeJsonPayload(Json::Value&, org::apache::nifi::minifi::c2::C2Payload const&)':
    RESTProtocol.cpp:(.text+0x27c2): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x286c): undefined reference to `Json::Value::isMember(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&) const'
    RESTProtocol.cpp:(.text+0x28a1): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x28cf): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x293b): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x2b0b): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x2b25): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x2b6c): undefined reference to `Json::Value::Value(std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x2b85): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    RESTProtocol.cpp:(.text+0x2c02): undefined reference to `Json::Value::operator[](std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char> > const&)'
    collect2: error: ld returned 1 exit status
    make[2]: *** [SiteToSiteRestTest] Error 1
    make[1]: *** [CMakeFiles/SiteToSiteRestTest.dir/all] Error 2
    make: *** [all] Error 2
    ```


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by benqiu2016 <gi...@git.apache.org>.
Github user benqiu2016 commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137857666
  
    --- Diff: libminifi/include/ResourceClaim.h ---
    @@ -55,23 +58,20 @@ class ResourceClaim : public std::enable_shared_from_this<ResourceClaim> {
     
       ResourceClaim(const std::string path, std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, bool deleted = false);
       // Destructor
    -  virtual ~ResourceClaim() {
    +  ~ResourceClaim() {
    --- End diff --
    
    why remove the virtual?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137861356
  
    --- Diff: libminifi/include/c2/HeartBeatReporter.h ---
    @@ -0,0 +1,99 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_HEARTBEATREPORTER_H_
    +#define LIBMINIFI_INCLUDE_C2_HEARTBEATREPORTER_H_
    +
    +#include "C2Protocol.h"
    +#include "C2Payload.h"
    +#include "core/controller/ControllerServiceProvider.h"
    +#include "properties/Configure.h"
    +#include "core/Connectable.h"
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +/**
    + * Defines a heart beat reporting interface. Note that this differs from
    + * C2Protocol as heartbeats can be any interface which provides only one way communication.
    + */
    +class HeartBeatReporter : public core::Connectable {
    + public:
    +
    +  HeartBeatReporter(std::string name, uuid_t uuid)
    +      : core::Connectable(name, uuid), controller_(nullptr), configuration_(nullptr) {
    +  }
    +
    +  virtual void initialize(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<Configure> &configure){
    +    controller_ = controller;
    +    configuration_ = configure;
    +  }
    +  virtual ~HeartBeatReporter() {
    +  }
    +  /**
    +   * Send a C2 payload to the provided URI. The direction indicates to the protocol whether or not this a transmit or receive operatin.
    --- End diff --
    
    Minor typo.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi-minifi-cpp/pull/134


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137585243
  
    --- Diff: libminifi/include/core/state/metrics/MetricsListener.h ---
    @@ -0,0 +1,131 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_METRICS_H_
    +#define LIBMINIFI_INCLUDE_C2_METRICS_H_
    +
    +#include <vector>
    +
    +#include "MetricsBase.h"
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +namespace metrics {
    +
    +/**
    + * Purpose: Class that will represent the
    --- End diff --
    
    Incompete comment


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137861761
  
    --- Diff: libminifi/include/c2/protocols/RESTProtocol.h ---
    @@ -0,0 +1,74 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_PROTOCOLS_RESTOPERATIONS_H_
    +#define LIBMINIFI_INCLUDE_C2_PROTOCOLS_RESTOPERATIONS_H_
    --- End diff --
    
    Inconsistent include guard?


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    We should probably have some docs for the C2 API.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137811526
  
    --- Diff: libminifi/include/FlowController.h ---
    @@ -124,14 +123,33 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
       virtual bool isRunning() {
         return running_.load();
       }
    +
       // Whether the Flow Controller has already been initialized (loaded flow XML)
       virtual bool isInitialized() {
         return initialized_.load();
       }
       // Start to run the Flow Controller which internally start the root process group and all its children
    -  virtual bool start();
    +  virtual int16_t start();
    +  virtual int16_t pause() {
    +    return -1;
    +  }
       // Unload the current flow YAML, clean the root process group and all its children
    -  virtual void stop(bool force);
    +  virtual int16_t stop(bool force, uint64_t timeToWait = 0);
    +  virtual int16_t applyUpdate(const std::string &configuration);
    +  virtual int16_t drainRepositories() {
    +
    +    return -1;
    +  }
    +
    +  virtual std::vector<std::shared_ptr<state::StateController>> getComponents(const std::string &name);
    +
    +  virtual std::vector<std::shared_ptr<StateController>> getAllComponents();
    --- End diff --
    
    Missing state:: prefix on StateController type makes this signature declaration inconsistent with its implementation/other uses. Mostly cosmetic?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r139965480
  
    --- Diff: libminifi/include/core/Processor.h ---
    @@ -212,19 +217,35 @@ class Processor : public Connectable, public ConfigurableComponent, public std::
      public:
     
       // OnTrigger method, implemented by NiFi Processor Designer
    +  virtual void onTrigger(std::shared_ptr<ProcessContext> context, std::shared_ptr<ProcessSession> session){
    +    onTrigger(context.get(),session.get());
    +  }
       virtual void onTrigger(ProcessContext *context, ProcessSession *session) = 0;
       // Initialize, overridden by NiFi Process Designer
       virtual void initialize() {
       }
       // Scheduled event hook, overridden by NiFi Process Designer
    +  virtual void onSchedule(std::shared_ptr<ProcessContext> context, std::shared_ptr<ProcessSessionFactory> sessionFactory){
    +    onSchedule(context.get(),sessionFactory.get());
    +  }
       virtual void onSchedule(ProcessContext *context, ProcessSessionFactory *sessionFactory) {
       }
     
       // Check all incoming connections for work
       bool isWorkAvailable();
     
    +  void setStreamFactory(std::shared_ptr<minifi::io::StreamFactory> stream_factory) {
    --- End diff --
    
    There is a use case in the event that C2 were to change the communication mechanism and only the communication mechanism we can do that without updating the entire flow. That can be tantamount to changing from TLS to non secure sockets or vice versa if the need arises. I didn't include those capabilities in this PR but played with it...and that's something we should discuss globally. That may not be a capability we want to support, but it's one that was previously discussed. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137835890
  
    --- Diff: libminifi/include/ResourceClaim.h ---
    @@ -83,13 +83,19 @@ class ResourceClaim : public std::enable_shared_from_this<ResourceClaim> {
       }
     
       void deleteClaim() {
    -    if (!deleted_)
    -    {
    +    if (!deleted_) {
           deleted_ = true;
         }
     
       }
     
    +  bool exists(){
    +    if (claim_manager_ == nullptr){
    +      return false;
    --- End diff --
    
    This could lead to subtle bugs where a programming error (not setting/initializing claim_manager_) would appear as a runtime state or error condition (file lost/doesn't exist).


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    Also getting a test failure on CentOS 7.3:
    
    ```
    The following tests FAILED:
             19 - SocketTests (Failed)
    ```


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137859201
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    +
    +  /**
    +   * Returns raw data.
    +   */
    +  std::string getRawData() const;
    +
    +  /**
    +   * Add a nested payload.
    +   * @param payload payload to move into this object.
    +   */
    +  void addPayload(const C2Payload &&payload);
    +  /**
    +   * Get nested payloads.
    +   */
    +  const std::vector<C2Payload> &getNestedPayloads() const;
    +
    +  C2Payload &operator=(const C2Payload &&other);
    +  C2Payload &operator=(const C2Payload &other);
    +
    + protected:
    +
    +  // identifier for this payload.
    +  std::string ident_;
    +
    +  std::string label_;
    +
    +  std::vector<C2Payload> payloads_;
    +
    +  std::vector<C2ContentResponse> content_;
    +
    +  Operation op_;
    +
    +  bool raw_;
    +
    +  std::string raw_data_;
    +
    +  bool isResponse;
    +
    +};
    +
    +} /* namesapce c2 */
    +} /* namespace minifi */
    +} /* namespace nifi */
    +} /* namespace apache */
    +} /* namespace org */
    +
    +#endif /* LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_ */
    --- End diff --
    
    Yes, the protocol can implement these formats, and is a natural extension point. Internally it didn't seem necessary to use those formats and incur costs of additional libraries for the payload. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137587193
  
    --- Diff: libminifi/include/core/state/metrics/RepositoryMetrics.h ---
    @@ -0,0 +1,101 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_CORE_STATE_METRICS_RepositoryMetrics_H_
    +#define LIBMINIFI_INCLUDE_CORE_STATE_METRICS_RepositoryMetrics_H_
    --- End diff --
    
    Inconsistent include guard capitalization.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by minifirocks <gi...@git.apache.org>.
Github user minifirocks commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @phrocker looks good
    @apiri please review and let me know whether you need me to merge the same. 


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    Taking a look...


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137860497
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    --- End diff --
    
    What was the reasoning behind making raw data a string vs. a set of bytes?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137812730
  
    --- Diff: libminifi/include/FlowController.h ---
    @@ -124,14 +123,33 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
       virtual bool isRunning() {
         return running_.load();
       }
    +
       // Whether the Flow Controller has already been initialized (loaded flow XML)
       virtual bool isInitialized() {
         return initialized_.load();
       }
       // Start to run the Flow Controller which internally start the root process group and all its children
    -  virtual bool start();
    +  virtual int16_t start();
    --- End diff --
    
    For cases where we are overriding a member function already declared virtual in the base class, the virtual keyword is redundant. For this case and others like it, it would be better to use the override keyword so the compiler will check that the function is indeed overridden as expected.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by apiri <gi...@git.apache.org>.
Github user apiri commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    License, code, and functionality looks good.  Build and operation functions as expected over an extended run time over the weekend.  Will get this beast of a PR merged in.



---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137865874
  
    --- Diff: libminifi/include/core/state/UpdateController.h ---
    @@ -0,0 +1,255 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +#define LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +
    +#include <string>
    +#include "utils/ThreadPool.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +
    +enum class UpdateState {
    +  INITIATE,
    +  FULLY_APPLIED,
    +  READ_COMPLETE,
    +  PARTIALLY_APPLIED,
    +  NOT_APPLIED,
    +  SET_ERROR,
    +  READ_ERROR,
    +  NESTED  // multiple updates embedded into one
    +
    +};
    +
    +/**
    + * Represents the status of an update operation.
    + *
    + */
    +class UpdateStatus {
    + public:
    +  UpdateStatus(UpdateState state, int16_t reason = 0);
    +
    +  UpdateStatus(const UpdateStatus &other);
    +
    +  UpdateStatus(const UpdateStatus &&other);
    +
    +  UpdateState getState() const;
    +
    +  std::string getError() const;
    +
    +  int16_t getReadonCode() const;
    +
    +  UpdateStatus &operator=(const UpdateStatus &&other);
    +
    +  UpdateStatus &operator=(const UpdateStatus &other);
    +   private:
    +  UpdateState state_;
    +  std::string error_;
    +  int16_t reason_;
    +};
    +
    +class Update
    +{
    --- End diff --
    
    Bracket placement inconsistency. Not sure which one is right. Might be a great time to run clang format or the like with this big merge.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137886296
  
    --- Diff: libminifi/include/core/state/UpdateController.h ---
    @@ -0,0 +1,255 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +#define LIBMINIFI_INCLUDE_C2_UPDATECONTROLLER_H_
    +
    +#include <string>
    +#include "utils/ThreadPool.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace state {
    +
    +enum class UpdateState {
    +  INITIATE,
    +  FULLY_APPLIED,
    +  READ_COMPLETE,
    +  PARTIALLY_APPLIED,
    +  NOT_APPLIED,
    +  SET_ERROR,
    +  READ_ERROR,
    +  NESTED  // multiple updates embedded into one
    +
    +};
    +
    +/**
    + * Represents the status of an update operation.
    + *
    + */
    +class UpdateStatus {
    + public:
    +  UpdateStatus(UpdateState state, int16_t reason = 0);
    +
    +  UpdateStatus(const UpdateStatus &other);
    +
    +  UpdateStatus(const UpdateStatus &&other);
    +
    +  UpdateState getState() const;
    +
    +  std::string getError() const;
    +
    +  int16_t getReadonCode() const;
    +
    +  UpdateStatus &operator=(const UpdateStatus &&other);
    +
    +  UpdateStatus &operator=(const UpdateStatus &other);
    +   private:
    +  UpdateState state_;
    +  std::string error_;
    +  int16_t reason_;
    +};
    +
    +class Update
    +{
    + public:
    +
    +  Update(UpdateStatus status)
    +      : status_(status) {
    +
    +  }
    +
    +  Update(const Update &other)
    +      : status_(other.status_) {
    +
    +  }
    +
    +  Update(const Update &&other)
    +      : status_(std::move(other.status_)) {
    +
    +  }
    +
    +  virtual ~Update() {
    +
    +  }
    +
    +  virtual bool validate() {
    +    return true;
    +  }
    +
    +  const UpdateStatus &getStatus() const {
    +    return status_;
    +  }
    +
    +  Update &operator=(const Update &&other) {
    +    status_ = std::move(other.status_);
    +    return *this;
    +  }
    +
    +  Update &operator=(const Update &other) {
    +    status_ = other.status_;
    +    return *this;
    +  }
    +
    + protected:
    +  UpdateStatus status_;
    +};
    +
    +/**
    + * Justification and Purpose: Update Runner reflects the post execution functors that determine if
    + * a given function that is running within a thread pool worker needs to end.
    + *
    + * Design: Simply implements isFinished and isCancelled, which it receives by way of the AfterExecute
    + * class.
    + */
    +class UpdateRunner : public utils::AfterExecute<Update>
    +{
    + public:
    +  explicit UpdateRunner(std::atomic<bool> &running)
    +      : running_(&running) {
    +  }
    +
    +  explicit UpdateRunner(UpdateRunner && other)
    +      : running_(std::move(other.running_)) {
    +
    +  }
    +
    +  ~UpdateRunner() {
    +
    +  }
    +
    +  virtual bool isFinished(const Update &result) {
    +    if ((result.getStatus().getState() == UpdateState::FULLY_APPLIED || result.getStatus().getState() == UpdateState::READ_COMPLETE) && *running_) {
    --- End diff --
    
    Did notice the producer/consumer in c2agent.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    I noted this in the review above, but I think this would also generally be a good time to run a clang format/clang tidy as this change spans a large amount of code. Would put us in a nice place going forward with smaller, more focused commits.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by benqiu2016 <gi...@git.apache.org>.
Github user benqiu2016 commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137859968
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    +
    +  /**
    +   * Returns raw data.
    +   */
    +  std::string getRawData() const;
    +
    +  /**
    +   * Add a nested payload.
    +   * @param payload payload to move into this object.
    +   */
    +  void addPayload(const C2Payload &&payload);
    +  /**
    +   * Get nested payloads.
    +   */
    +  const std::vector<C2Payload> &getNestedPayloads() const;
    +
    +  C2Payload &operator=(const C2Payload &&other);
    +  C2Payload &operator=(const C2Payload &other);
    +
    + protected:
    +
    +  // identifier for this payload.
    +  std::string ident_;
    +
    +  std::string label_;
    +
    +  std::vector<C2Payload> payloads_;
    +
    +  std::vector<C2ContentResponse> content_;
    +
    +  Operation op_;
    +
    +  bool raw_;
    +
    +  std::string raw_data_;
    +
    +  bool isResponse;
    +
    +};
    +
    +} /* namesapce c2 */
    +} /* namespace minifi */
    +} /* namespace nifi */
    +} /* namespace apache */
    +} /* namespace org */
    +
    +#endif /* LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_ */
    --- End diff --
    
    do we support version field in the payload/metric to handle backward compatible.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137881550
  
    --- Diff: libminifi/src/RemoteProcessorGroupPort.cpp ---
    @@ -221,51 +223,28 @@ void RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo() {
     
       if (!rest_user_name_.empty()) {
         std::string loginUrl = this->protocol_ + this->host_ + ":" + std::to_string(this->port_) + "/nifi-api/access/token";
    -    token = utils::get_token(loginUrl, this->rest_user_name_, this->rest_password_, this->securityConfig_);
    -    logger_->log_debug("Token from NiFi REST Api endpoint %s", token);
    +    utils::HTTPClient client(loginUrl, ssl_service);
    --- End diff --
    
    +1 for stripping raw curl and adding HTTPClient.


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137858594
  
    --- Diff: libminifi/include/ResourceClaim.h ---
    @@ -55,23 +58,20 @@ class ResourceClaim : public std::enable_shared_from_this<ResourceClaim> {
     
       ResourceClaim(const std::string path, std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, bool deleted = false);
       // Destructor
    -  virtual ~ResourceClaim() {
    +  ~ResourceClaim() {
    --- End diff --
    
    That class is not extended and there are no other virtual functions, thus we were creating a virtual table for this. There appears to be no intent to extend it so the performance impact seemed unnecessary. 


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    @achristianson is the centos 7 a fresh build and did you install leveldb? I've seen that error but when we installed leveldb the error went away and we were able to run tests and ran about 78 instances on ec2. Let me know if you are running from a fresh install...if not I'll spawn some more instances to debug. Thanks


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137824349
  
    --- Diff: libminifi/include/RemoteProcessorGroupPort.h ---
    @@ -43,14 +44,15 @@ class RemoteProcessorGroupPort : public core::Processor {
       /*!
        * Create a new processor
        */
    -  RemoteProcessorGroupPort(const std::shared_ptr<io::StreamFactory> &stream_factory, std::string name, std::string url, std::shared_ptr<Configure> configure, uuid_t uuid = nullptr)
    +  RemoteProcessorGroupPort(const std::shared_ptr<io::StreamFactory> &stream_factory, std::string name, std::string url, const std::shared_ptr<Configure> &configure, uuid_t uuid = nullptr)
           : core::Processor(name, uuid),
             configure_(configure),
             direction_(SEND),
             transmitting_(false),
    +        timeout_(0),
             logger_(logging::LoggerFactory<RemoteProcessorGroupPort>::getLogger()),
             url_(url),
    -        securityConfig_(configure) {
    +        ssl_service(std::make_shared<controllers::SSLContextService>("RemoteProcessorGroupPortSSLContextService", configure)) {
    --- End diff --
    
    Should we be using a ControllerServiceProvider here rather than constructing the SSLContextService directly?


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r139694071
  
    --- Diff: libminifi/include/FlowController.h ---
    @@ -124,14 +123,33 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
       virtual bool isRunning() {
         return running_.load();
       }
    +
       // Whether the Flow Controller has already been initialized (loaded flow XML)
       virtual bool isInitialized() {
         return initialized_.load();
       }
       // Start to run the Flow Controller which internally start the root process group and all its children
    -  virtual bool start();
    +  virtual int16_t start();
    --- End diff --
    
    I completely agree. I've created a ticket to track this: https://issues.apache.org/jira/browse/MINIFI-400 MINIFI-399 will encompass other structural changes that we can apply


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r139699391
  
    --- Diff: libminifi/include/c2/C2Payload.h ---
    @@ -0,0 +1,187 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PAYLOAD_H_
    +
    +#include <memory>
    +#include <string>
    +#include <map>
    +#include "core/state/UpdateController.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +enum Operation {
    +  ACKNOWLEDGE,
    +  START,
    +  STOP,
    +  RESTART,
    +  DESCRIBE,
    +  HEARTBEAT,
    +  UPDATE,
    +  VALIDATE,
    +  CLEAR
    +};
    +
    +enum Direction {
    +  TRANSMIT,
    +  RECEIVE
    +};
    +
    +class C2ContentResponse {
    + public:
    +  C2ContentResponse(Operation op);
    +
    +  C2ContentResponse(const C2ContentResponse &other);
    +
    +  C2ContentResponse(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &&other);
    +
    +  C2ContentResponse & operator=(const C2ContentResponse &other);
    +
    +  Operation op;
    +  // determines if the operation is required
    +  bool required;
    +  // identifier
    +  std::string ident;
    +  // delay before running
    +  uint32_t delay;
    +  // max time before this response will no longer be honored.
    +  uint64_t ttl;
    +  // name applied to commands
    +  std::string name;
    +  // commands that correspond with the operation.
    +  std::map<std::string, std::string> operation_arguments;
    +//  std::vector<std::string> content;
    +};
    +
    +/**
    + * C2Payload is an update for the state manager.
    + * Note that the payload can either consist of other payloads or
    + * have content directly within it, represented by C2ContentResponse objects, above.
    + *
    + * Payloads can also contain raw data, which can be binary data.
    + */
    +class C2Payload : public state::Update {
    + public:
    +  virtual ~C2Payload() {
    +
    +  }
    +
    +  C2Payload(Operation op, std::string identifier, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
    +
    +  C2Payload(const C2Payload &other);
    +
    +  C2Payload(const C2Payload &&other);
    +
    +  void setIdentifier(const std::string &ident);
    +
    +  std::string getIdentifier() const;
    +
    +  void setLabel(const std::string label) {
    +    label_ = label;
    +  }
    +
    +  std::string getLabel() const {
    +    return label_;
    +  }
    +
    +  /**
    +   * Gets the operation for this payload. May be nested or a single operation.
    +   */
    +  Operation getOperation() const;
    +
    +  /**
    +   * Validate the payload, if necessary and/or possible.
    +   */
    +  virtual bool validate();
    +
    +  /**
    +   * Get content responses from this payload.
    +   */
    +  const std::vector<C2ContentResponse> &getContent() const;
    +
    +  /**
    +   * Add a content response to this payload.
    +   */
    +  void addContent(const C2ContentResponse &&content);
    +
    +  /**
    +   * Determines if this object contains raw data.
    +   */
    +  bool isRaw() const;
    +
    +  /**
    +   * Sets raw data within this object.
    +   */
    +  void setRawData(const std::string &data);
    --- End diff --
    
    A string contains raw bytes. It can be accessed with std::string::data. I don't know if there is really a fundamental difference between vector<char> and std::string. Performance is negligible and a setRawData(vector<char> data ) exists, so we don't impose a semantic that says you must use text, if the developer interprets the signature as such. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137861024
  
    --- Diff: libminifi/include/c2/C2Protocol.h ---
    @@ -0,0 +1,119 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2PROTOCOL_H_
    +#define LIBMINIFI_INCLUDE_C2_C2PROTOCOL_H_
    +
    +#include "C2Payload.h"
    +#include "core/controller/ControllerServiceProvider.h"
    +#include "properties/Configure.h"
    +#include "core/Connectable.h"
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +/**
    + * Defines a protocol to perform state management of the minifi agent.
    + */
    +class C2Protocol : public core::Connectable {
    + public:
    +
    +  C2Protocol(std::string name, uuid_t uuid)
    +      : core::Connectable(name, uuid),
    +        running_(true) {
    +
    +  }
    +
    +  virtual void initialize(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller, const std::shared_ptr<Configure> &configure) {
    +    controller_ = controller;
    +    configuration_ = configure;
    +  }
    +  virtual ~C2Protocol() {
    +
    +  }
    +
    +  /**
    +   * Update the configuration.
    +   */
    +  virtual void update(const std::shared_ptr<Configure> &configure)=0;
    +
    +  /**
    +   * Send a C2 payload to the provided URI. The direction indicates to the protocol whether or not this a transmit or receive operatin.
    +   * Depending on the protocol this may mean different things.
    +   *
    +   * @param url url.
    +   * @param operation payload to perform and/or send
    +   * @param direction direction of the C2 operation.
    +   * @param async whether or not this is an asynchronous operation
    +   * @return payload from the response or a response to come back in the face of an asynchronous operation.
    +   */
    +  virtual C2Payload consumePayload(const std::string &url, const C2Payload &operation, Direction direction = TRANSMIT, bool async = false) = 0;
    +
    +  /**
    +   * Send a C2 payload . The direction indicates to the protocol whether or not this a transmit or receive operatin.
    --- End diff --
    
    Minor typo.


---

[GitHub] nifi-minifi-cpp issue #134: MINIFI-339: Add C2 base allowing for 1 protocol ...

Posted by apiri <gi...@git.apache.org>.
Github user apiri commented on the issue:

    https://github.com/apache/nifi-minifi-cpp/pull/134
  
    reviewing


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r141920195
  
    --- Diff: libminifi/include/c2/C2Agent.h ---
    @@ -0,0 +1,203 @@
    +/**
    + *
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +#ifndef LIBMINIFI_INCLUDE_C2_C2AGENT_H_
    +#define LIBMINIFI_INCLUDE_C2_C2AGENT_H_
    +
    +#include <utility>
    +#include <functional>
    +#include <future>
    +#include <memory>
    +#include <mutex>
    +#include <thread>
    +#include "core/state/UpdateController.h"
    +#include "core/state/metrics/MetricsBase.h"
    +#include "C2Payload.h"
    +#include "C2Protocol.h"
    +#include "io/validation.h"
    +#include "protocols/Protocols.h"
    +
    +namespace org {
    +namespace apache {
    +namespace nifi {
    +namespace minifi {
    +namespace c2 {
    +
    +/**
    + * Purpose and Justification: C2 agent will be the mechanism that will abstract the protocol to do the work.
    + *
    + * The protocol represents a transformation layer into the objects seen in C2Payload. That transformation may
    + * be minimal or extreme, depending on the protocol itself.
    + *
    + * Metrics Classes defined here:
    + *
    + *   0 HeartBeat --  RESERVED
    + *   1-255 Defined by the configuration file.
    + */
    +class C2Agent : public state::UpdateController, public state::metrics::MetricsSink, public std::enable_shared_from_this<C2Agent> {
    + public:
    +
    +  C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvider> &controller,
    +          const std::shared_ptr<state::StateMonitor> &updateSink,
    +          const std::shared_ptr<Configure> &configure);
    +
    +  virtual ~C2Agent() {
    +
    +  }
    +
    +  /**
    +   * Sends the heartbeat to ths server. Will include metrics
    +   * in the payload if they exist.
    +   */
    +  void performHeartBeat();
    +
    +  virtual std::vector<std::function<state::Update()>> getFunctions() {
    +    return functions_;
    +  }
    +
    +  /**
    +   * Sets the metric within this sink
    +   * @param metric metric to set
    +   * @param return 0 on success, -1 on failure.
    +   */
    +  virtual int16_t setMetrics(const std::shared_ptr<state::metrics::Metrics> &metric);
    +
    + protected:
    +
    +  /**
    +   * Configure the C2 agent
    +   */
    +  void configure(const std::shared_ptr<Configure> &configure, bool reconfigure = true);
    +
    +  /**
    +   * Serializes metrics into a payload.
    +   * @parem parent_paylaod parent payload into which we insert the newly generated payload.
    +   * @param name name of this metric
    +   * @param metrics metrics to include.
    +   */
    +  void serializeMetrics(C2Payload &parent_payload, const std::string &name, const std::vector<state::metrics::MetricResponse> &metrics);
    +
    +  /**
    +   * Extract the payload
    +   * @param resp payload to be moved into the function.
    +   */
    +  void extractPayload(const C2Payload &&resp);
    +
    +  /**
    +   * Extract the payload
    +   * @param payload reference.
    +   */
    +  void extractPayload(const C2Payload &resp);
    +
    +  /**
    +   * Enqueues a C2 server response for us to evaluate and parse.
    +   */
    +  void enqueue_c2_server_response(C2Payload &&resp) {
    +    std::lock_guard<std::timed_mutex> lock(queue_mutex);
    +    responses.push_back(std::move(resp));
    --- End diff --
    
    We want to maintain ordering, which can't be guaranteed with a lock free queue. 


---

[GitHub] nifi-minifi-cpp pull request #134: MINIFI-339: Add C2 base allowing for 1 pr...

Posted by achristianson <gi...@git.apache.org>.
Github user achristianson commented on a diff in the pull request:

    https://github.com/apache/nifi-minifi-cpp/pull/134#discussion_r137863977
  
    --- Diff: libminifi/include/core/Processor.h ---
    @@ -153,7 +154,8 @@ class Processor : public Connectable, public ConfigurableComponent, public std::
       }
       // decrement Active Task Counts
       void decrementActiveTask(void) {
    -    active_tasks_--;
    +    if (active_tasks_ > 0)
    +      active_tasks_--;
    --- End diff --
    
    One has to wonder how we could get into this condition in the first place. Is there a bug/race somewhere else?


---