You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/11/29 12:16:25 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

martinzink opened a new pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219


   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] 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 GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778826634



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -60,6 +60,12 @@ def start_kafka_broker(self):
         self.cluster.deploy('kafka-broker')
         assert self.wait_for_container_startup_to_finish('kafka-broker')
 
+    def start_splunk(self):
+        self.cluster.acquire_container('splunk', 'splunk')
+        self.cluster.deploy('splunk')
+        assert self.wait_for_container_startup_to_finish('splunk')
+        assert self.cluster.enable_hec_indexer('splunk', 'splunk_hec_token')

Review comment:
       I think its the former, if minifi starts up before splunk is fully operational, then GetFile can ingest the input and pass it to PutSplunkHTTP which will fail due to the unreachable splunk hec.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780321716



##########
File path: docker/test/integration/features/splunk.feature
##########
@@ -0,0 +1,24 @@
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: A MiNiFi instance transfers data to a Splunk HEC

Review comment:
       Thanks, looks great!

##########
File path: docker/requirements.txt
##########
@@ -6,3 +6,4 @@ confluent-kafka==1.7.0
 PyYAML==5.4.1
 m2crypto==0.37.1
 watchdog==2.1.2
+pyopenssl

Review comment:
       Please specify version number of the package to avoid possible future problems with newer versions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781271742



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));

Review comment:
       thats a valid point, in that case I think every flowfile with duplicated ack_id should be transferred to failure (because even if the server acknowledges we cant be sure which one is it)
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-a2db2ff59dd1ebf5f1e3e053781c55708df909e3009ee21015da3fe04218def4R109-R119
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781040629



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));

Review comment:
       not sure if it is intentional but `addAttribute` only adds the attribute if it does not exist on the flowfile




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781285637



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {

Review comment:
       The TimePeriodValue is being refactored in #1225 
   I think this will merge sooner than that, how about I fix this issue there?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777581938



##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -37,6 +37,24 @@ namespace minifi {
 namespace utils {
 namespace timeutils {
 
+/**
+ * Converts the time point to the elapsed time since epoch
+ * @returns TimeUnit since epoch
+ */
+template<typename TimeUnit, typename TimePoint>
+uint64_t getTimestamp(const TimePoint& time_point) {
+  return std::chrono::duration_cast<TimeUnit>(time_point.time_since_epoch()).count();
+}
+
+/**
+ * Converts the time since epoch into a time point
+ * @returns the time point matching the input timestamp
+ */
+template<typename TimeUnit, typename ClockType>
+std::chrono::time_point<ClockType> getTimePoint(uint64_t timestamp) {
+  return std::chrono::time_point<ClockType>() + TimeUnit(timestamp);
+}

Review comment:
       I criticized these in your other PR, which would apply here, too. If you think that they are useful, feel free to reintroduce them there, but if they don't improve the readability, you may want to consider removing them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781281741



##########
File path: PROCESSORS.md
##########
@@ -50,7 +50,9 @@
 - [PutS3Object](#puts3object)
 - [PutSFTP](#putsftp)
 - [PutSQL](#putsql)
+- [PutSplunkHTTP](#putsplunkhttp)

Review comment:
       Reordered it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-fd2410931e7fdc4bf8b3ce23f5f7a27c7aacdf9337320626d86d806448c90b9bL52

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));

Review comment:
       Good catch, it wasnt intentional. I very well can image situations where we wanna retry after a failed PutSplunkHTTP.
   fixed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR122-R123

##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());

Review comment:
       :+1:  done https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-a2db2ff59dd1ebf5f1e3e053781c55708df909e3009ee21015da3fe04218def4R40

##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());

Review comment:
       :+1: done https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-a2db2ff59dd1ebf5f1e3e053781c55708df909e3009ee21015da3fe04218def4R45




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778978494



##########
File path: extensions/splunk/tests/PutSplunkHTTPTests.cpp
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "PutSplunkHTTP.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "MockSplunkHEC.h"
+
+using PutSplunkHTTP = org::apache::nifi::minifi::extensions::splunk::PutSplunkHTTP;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+
+
+TEST_CASE("PutSplunkHTTP tests", "[putsplunkhttp]") {
+  MockSplunkHEC mock_splunk_hec("10133");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto put_splunk_http = std::dynamic_pointer_cast<PutSplunkHTTP>(plan->addProcessor("PutSplunkHTTP", "put_splunk_http"));
+  auto read_from_success = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_success"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, put_splunk_http);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Success, read_from_success);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Failure, read_from_failure);
+
+  read_from_success->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Hostname.getName(), "localhost");
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Happy path") {

Review comment:
       Added tests for checking the content-type 
   it was actually bugged due to a typo so thanks for suggesting the test :+1: 
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/ee52021fd2a471abb9504282a4bddf2591186cb6




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780315866



##########
File path: docker/test/integration/features/splunk.feature
##########
@@ -0,0 +1,24 @@
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: A MiNiFi instance transfers data to a Splunk HEC

Review comment:
       Good idea, I've added a docker test where SSL is enabled on both minifi and splunk in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/2cec29069f5d2f4e4bbb062d38eaf6742483ff00




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780322404



##########
File path: docker/requirements.txt
##########
@@ -6,3 +6,4 @@ confluent-kafka==1.7.0
 PyYAML==5.4.1
 m2crypto==0.37.1
 watchdog==2.1.2
+pyopenssl

Review comment:
       Please specify version number of the package to avoid possible future problems with newer versions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780314709



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409

##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409

##########
File path: docker/test/integration/features/splunk.feature
##########
@@ -0,0 +1,24 @@
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: A MiNiFi instance transfers data to a Splunk HEC

Review comment:
       Good idea, I've added a docker test where SSL is enabled on both minifi and splunk in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/2cec29069f5d2f4e4bbb062d38eaf6742483ff00

##########
File path: docker/requirements.txt
##########
@@ -6,3 +6,4 @@ confluent-kafka==1.7.0
 PyYAML==5.4.1
 m2crypto==0.37.1
 watchdog==2.1.2
+pyopenssl

Review comment:
       specified it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/e77a2891da3ba880f1a24b8a74757d572dc2ccde




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781900003



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {

Review comment:
       good idea




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778840826



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {
+  session.read(flow_file, &payload_callback);
+  payload_callback_obj.ptr = &payload_callback;
+  payload_callback_obj.pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(&payload_callback_obj);
+  client.setSeekFunction(&payload_callback_obj);
+
+  auto content_type = getContentType(context, *flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}

Review comment:
       good idea, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR143




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r782082669



##########
File path: extensions/http-curl/client/HTTPClient.cpp
##########
@@ -212,7 +212,10 @@ void HTTPClient::setContentType(std::string content_type) {
 }
 
 std::string HTTPClient::escape(std::string string_to_escape) {
-  return curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()));
+  char* escaped_chars = curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()));
+  std::string escaped_string(escaped_chars);
+  curl_free(escaped_chars);
+  return escaped_string;

Review comment:
       In theory string's constructor can throw which would result in leaking `escaped_chars`. Consider using `unique_ptr` with a custom curl deleter.
   In practice it will never fail to allocate on linux, but will rather block indefinitely until there is an available page. On Windows, bad_alloc is possible.
   ```suggestion
     struct curl_deleter { void operator()(void* p) noexcept { curl_free(p); } };
     std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
     std::string escaped_string(escaped_chars.get());
     return escaped_string;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r767729620



##########
File path: extensions/splunk/PutSplunkHTTP.h
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <memory>
+
+#include "SplunkHECProcessor.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+class PutSplunkHTTP final : public SplunkHECProcessor {

Review comment:
       Please specify `InputRequirements` by overriding `getInputRequirements`.

##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  updateSupportedProperties({MaximumWaitingTime, MaxQuerySize});
+}

Review comment:
       There may be value in listing all of the supported properties here instead of touching the set twice. It's definitely more declarative style, which makes reasoning easier IMO.
   Related discussion: https://github.com/apache/nifi-minifi-cpp/pull/1158#discussion_r710018162

##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -37,6 +37,24 @@ namespace minifi {
 namespace utils {
 namespace timeutils {
 
+/**
+ * Converts the time point to the elapsed time since epoch
+ * @returns TimeUnit since epoch
+ */
+template<typename TimeUnit, typename TimePoint>
+uint64_t getTimeStamp(const TimePoint& time_point) {

Review comment:
       Timestamp is used as a single word in the majority of cases, so it should be capitalized as such. 
   ```suggestion
   uint64_t getTimestamp(const TimePoint& time_point) {
   ```

##########
File path: extensions/splunk/SplunkHECProcessor.h
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+
+#pragma once
+#include <string>
+#include <memory>
+
+#include "core/Processor.h"
+
+
+namespace org::apache::nifi::minifi::utils {
+class HTTPClient;
+}
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+class SplunkHECProcessor : public core::Processor {
+ public:
+  EXTENSIONAPI static const core::Property Hostname;
+  EXTENSIONAPI static const core::Property Port;
+  EXTENSIONAPI static const core::Property Token;
+  EXTENSIONAPI static const core::Property SplunkRequestChannel;
+  EXTENSIONAPI static const core::Property SSLContext;
+
+  explicit SplunkHECProcessor(const std::string& name, const utils::Identifier& uuid = {})
+      : Processor(name, uuid) {
+  }
+  ~SplunkHECProcessor() override {};

Review comment:
       ```suggestion
     ~SplunkHECProcessor() override = default;
   ```

##########
File path: docker/test/integration/minifi/core/SplunkContainer.py
##########
@@ -0,0 +1,26 @@
+import logging
+from .Container import Container
+
+
+class SplunkContainer(Container):
+    def __init__(self, name, vols, network, image_store):
+        super().__init__(name, 'splunk', vols, network, image_store)
+
+    def get_startup_finished_log_entry(self):
+        return "Ansible playbook complete, will begin streaming splunkd_stderr.log"
+
+    def deploy(self):
+        if not self.set_deployed():
+            return
+
+        logging.info('Creating and running Splunk docker container...')
+        self.client.containers.run(
+            self.image_store.get_image(self.get_engine()),
+            detach=True,
+            name=self.name,
+            network=self.network.name,
+            environment=[
+                "SPLUNK_START_ARGS=--accept-license",

Review comment:
       What license are we accepting here without a prompt? I found an Apache License 2.0 on their page, but it seems strange that they would require accepting an open source license in such a verbose way.

##########
File path: extensions/splunk/SplunkHECProcessor.cpp
##########
@@ -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.
+ */
+
+#include "SplunkHECProcessor.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property SplunkHECProcessor::Hostname(core::PropertyBuilder::createProperty("Hostname")
+    ->withDescription("The ip address or hostname of the Splunk server.")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Port(core::PropertyBuilder::createProperty("Port")
+    ->withDescription("The HTTP Event Collector HTTP Port Number.")
+    ->withDefaultValue("8088")->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Token(core::PropertyBuilder::createProperty("Token")
+    ->withDescription("HTTP Event Collector token starting with the string Splunk. For example \'Splunk 1234578-abcd-1234-abcd-1234abcd\'")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::SplunkRequestChannel(core::PropertyBuilder::createProperty("Splunk Request Channel")
+    ->withDescription("Identifier of the used request channel.")->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::SSLContext(core::PropertyBuilder::createProperty("SSL Context Service")
+    ->withDescription("The SSL Context Service used to provide client certificate "
+                      "information for TLS/SSL (https) connections.")
+    ->isRequired(false)->withExclusiveProperty("Remote URL", "^http:.*$")
+    ->asType<minifi::controllers::SSLContextService>()->build());
+
+void SplunkHECProcessor::initialize() {
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel});
+}
+
+void SplunkHECProcessor::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+  if (!context->getProperty(Hostname.getName(), hostname_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Hostname");
+
+  if (!context->getProperty(Port.getName(), port_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Port");
+
+  if (!context->getProperty(Token.getName(), token_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Token");
+
+  if (!context->getProperty(SplunkRequestChannel.getName(), request_channel_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get SplunkRequestChannel");
+}
+
+std::string SplunkHECProcessor::getUrl() const {
+  return hostname_ + ":" + port_;

Review comment:
       This is not a URL. You may want to add a protocol prefix or rename the function. (network location?)

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {

Review comment:
       A const ref to a flow file should be enough here. If not, then getAttribute needs fixing.
   ```suggestion
   std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
   ```

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file->getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimeStamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         const std::unique_ptr<utils::ByteInputCallBack>& payload_callback,
+                                         const std::unique_ptr<utils::HTTPUploadCallback>& payload_callback_obj) {
+  session.read(flow_file, payload_callback.get());
+  payload_callback_obj->ptr = payload_callback.get();
+  payload_callback_obj->pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(payload_callback_obj.get());
+  client.setSeekFunction(payload_callback_obj.get());
+
+  auto content_type = getContentType(context, flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}
+}  // namespace
+
+void PutSplunkHTTP::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session);
+
+  auto ff = session->get();
+  if (!ff) {
+    context->yield();
+    return;
+  }
+  auto flow_file = gsl::not_null(std::move(ff));
+
+  utils::HTTPClient client(getUrl() + getEndpoint(*context, flow_file), getSSLContextService(*context));
+  setHeaders(client);
+
+  std::unique_ptr<utils::ByteInputCallBack> payload_callback = std::make_unique<utils::ByteInputCallBack>();
+  std::unique_ptr<utils::HTTPUploadCallback> payload_callback_obj = std::make_unique<utils::HTTPUploadCallback>();

Review comment:
       Don't repeat the type after `make_unique`, use `auto` instead.
   ```suggestion
     const auto payload_callback = std::make_unique<utils::ByteInputCallBack>();
     const auto payload_callback_obj = std::make_unique<utils::HTTPUploadCallback>();
   ```

##########
File path: docker/Dockerfile
##########
@@ -113,8 +114,8 @@ RUN cmake -DSTATIC_BUILD= -DSKIP_TESTS=true -DENABLE_ALL="${ENABLE_ALL}" -DENABL
     -DENABLE_COAP="${ENABLE_COAP}" -DENABLE_SQL="${ENABLE_SQL}" -DENABLE_MQTT="${ENABLE_MQTT}" -DENABLE_PCAP="${ENABLE_PCAP}" \
     -DENABLE_LIBRDKAFKA="${ENABLE_LIBRDKAFKA}" -DENABLE_SENSORS="${ENABLE_SENSORS}" -DENABLE_USB_CAMERA="${ENABLE_USB_CAMERA}" \
     -DENABLE_TENSORFLOW="${ENABLE_TENSORFLOW}" -DENABLE_AWS="${ENABLE_AWS}" -DENABLE_BUSTACHE="${ENABLE_BUSTACHE}" -DENABLE_SFTP="${ENABLE_SFTP}" \
-    -DENABLE_OPENWSMAN="${ENABLE_OPENWSMAN}" -DENABLE_AZURE="${ENABLE_AZURE}" -DENABLE_NANOFI=${ENABLE_NANOFI} -DENABLE_SYSTEMD=OFF \

Review comment:
       extra space added here

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file->getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimeStamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         const std::unique_ptr<utils::ByteInputCallBack>& payload_callback,
+                                         const std::unique_ptr<utils::HTTPUploadCallback>& payload_callback_obj) {

Review comment:
       This function only needs references to the actual object. No need to force the caller to use unique_ptr, just use normal references instead.
   ```suggestion
                                            utils::ByteInputCallBack& payload_callback,
                                            utils::HTTPUploadCallback& payload_callback_obj) {
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777461218



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file->getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimeStamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         const std::unique_ptr<utils::ByteInputCallBack>& payload_callback,
+                                         const std::unique_ptr<utils::HTTPUploadCallback>& payload_callback_obj) {
+  session.read(flow_file, payload_callback.get());
+  payload_callback_obj->ptr = payload_callback.get();
+  payload_callback_obj->pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(payload_callback_obj.get());
+  client.setSeekFunction(payload_callback_obj.get());
+
+  auto content_type = getContentType(context, flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}
+}  // namespace
+
+void PutSplunkHTTP::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session);
+
+  auto ff = session->get();
+  if (!ff) {
+    context->yield();
+    return;
+  }
+  auto flow_file = gsl::not_null(std::move(ff));
+
+  utils::HTTPClient client(getUrl() + getEndpoint(*context, flow_file), getSSLContextService(*context));
+  setHeaders(client);
+
+  std::unique_ptr<utils::ByteInputCallBack> payload_callback = std::make_unique<utils::ByteInputCallBack>();
+  std::unique_ptr<utils::HTTPUploadCallback> payload_callback_obj = std::make_unique<utils::HTTPUploadCallback>();

Review comment:
       changed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR145

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file->getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimeStamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         const std::unique_ptr<utils::ByteInputCallBack>& payload_callback,
+                                         const std::unique_ptr<utils::HTTPUploadCallback>& payload_callback_obj) {

Review comment:
       good idea changed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR135

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Success, Failure});
+  updateSupportedProperties({Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {

Review comment:
       good idea changed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR73

##########
File path: docker/Dockerfile
##########
@@ -113,8 +114,8 @@ RUN cmake -DSTATIC_BUILD= -DSKIP_TESTS=true -DENABLE_ALL="${ENABLE_ALL}" -DENABL
     -DENABLE_COAP="${ENABLE_COAP}" -DENABLE_SQL="${ENABLE_SQL}" -DENABLE_MQTT="${ENABLE_MQTT}" -DENABLE_PCAP="${ENABLE_PCAP}" \
     -DENABLE_LIBRDKAFKA="${ENABLE_LIBRDKAFKA}" -DENABLE_SENSORS="${ENABLE_SENSORS}" -DENABLE_USB_CAMERA="${ENABLE_USB_CAMERA}" \
     -DENABLE_TENSORFLOW="${ENABLE_TENSORFLOW}" -DENABLE_AWS="${ENABLE_AWS}" -DENABLE_BUSTACHE="${ENABLE_BUSTACHE}" -DENABLE_SFTP="${ENABLE_SFTP}" \
-    -DENABLE_OPENWSMAN="${ENABLE_OPENWSMAN}" -DENABLE_AZURE="${ENABLE_AZURE}" -DENABLE_NANOFI=${ENABLE_NANOFI} -DENABLE_SYSTEMD=OFF \

Review comment:
       fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-f34da55ca08f1a30591d8b0b3e885bcc678537b2a9a4aadea4f190806b374ddcR117




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777462139



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  SplunkHECProcessor::initialize();
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  updateSupportedProperties({MaximumWaitingTime, MaxQuerySize});
+}

Review comment:
       Changed it in 
   
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-a2db2ff59dd1ebf5f1e3e053781c55708df909e3009ee21015da3fe04218def4R66 
   
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR64




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778832909



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {

Review comment:
       good catch, I gotta be more careful with renaming things in the last minute :+1: 
   fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR130-R134




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780314709



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409

##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"

Review comment:
       removed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/5021417766c08d4c98026e26dae85155ef28e409




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781898341



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);

Review comment:
       curl providing an utility is neat, but according to the [documentation](https://curl.se/libcurl/c/curl_easy_escape.html) the `char*` returned should be freed using `curl_free` (instead we initialize an `std::string` with it and leak it) could you fix that as well?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781024958



##########
File path: bstrp_functions.sh
##########
@@ -445,6 +446,7 @@ read_feature_options(){
     x) ToggleFeature AZURE_ENABLED ;;
     y) ToggleFeature SYSTEMD_ENABLED ;;
     z) ToggleFeature NANOFI_ENABLED ;;
+    aa) ToggleFeature SPLUNK_ENABLED ;;

Review comment:
       we should update the usage message printed (`Enter choice [ A - Z or 1-7 ]`) to reflect the addition of `aa` option




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781271504



##########
File path: extensions/splunk/tests/MockSplunkHEC.h
##########
@@ -0,0 +1,214 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+#include <CivetServer.h>
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+
+
+class MockSplunkHandler : public CivetHandler {
+ public:
+  explicit MockSplunkHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : token_(std::move(token)), assertions_(assertions) {
+  }
+
+  enum HeaderResult {
+    MissingAuth,
+    InvalidAuth,
+    MissingReqChannel,
+    HeadersOk
+  };
+
+  bool handlePost(CivetServer*, struct mg_connection *conn) override {
+    switch (checkHeaders(conn)) {
+      case MissingAuth:
+        return send401(conn);
+      case InvalidAuth:
+        return send403(conn);
+      case MissingReqChannel:
+        return send400(conn);
+      case HeadersOk:
+        return handlePostImpl(conn);
+    }
+    return false;
+  }
+
+  HeaderResult checkHeaders(struct mg_connection *conn) const {
+    const struct mg_request_info* req_info = mg_get_request_info(conn);
+    assertions_(req_info);
+    auto auth_header = std::find_if(std::begin(req_info->http_headers),
+                                    std::end(req_info->http_headers),
+                                    [](auto header) -> bool {return strcmp(header.name, "Authorization") == 0;});
+    if (auth_header == std::end(req_info->http_headers))
+      return MissingAuth;
+    if (strcmp(auth_header->value, token_.c_str()) != 0)
+      return InvalidAuth;
+
+    auto request_channel_header = std::find_if(std::begin(req_info->http_headers),
+                                               std::end(req_info->http_headers),
+                                               [](auto header) -> bool {return strcmp(header.name, "X-Splunk-Request-Channel") == 0;});
+
+    if (request_channel_header == std::end(req_info->http_headers))
+      return MissingReqChannel;
+    return HeadersOk;
+  }
+
+  bool send400(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Data channel is missing\",\"code\":10}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send401(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Token is required\",\"code\":2}";
+    mg_printf(conn, "HTTP/1.1 401 Unauthorized\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send403(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Invalid token\",\"code\":4}";
+    mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+ protected:
+  virtual bool handlePostImpl(struct mg_connection *conn) = 0;
+  std::string token_;
+  std::function<void(const struct mg_request_info *request_info)>& assertions_;
+};
+
+class RawCollectorHandler : public MockSplunkHandler {
+ public:
+  explicit RawCollectorHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : MockSplunkHandler(std::move(token), assertions) {}
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    constexpr const char * body = "{\"text\":\"Success\",\"code\":0,\"ackId\":808}";
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+};
+
+class AckIndexerHandler : public MockSplunkHandler {
+ public:
+  explicit AckIndexerHandler(std::string token, std::vector<uint64_t> indexed_events, std::function<void(const struct mg_request_info *request_info)>& assertions)
+      : MockSplunkHandler(std::move(token), assertions), indexed_events_(indexed_events) {}
+
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    std::vector<char> data;
+    data.reserve(2048);
+    mg_read(conn, data.data(), 2048);
+    rapidjson::Document post_data;
+
+    rapidjson::ParseResult parse_result = post_data.Parse<rapidjson::kParseStopWhenDoneFlag>(data.data());
+    if (parse_result.IsError())
+      return sendInvalidFormat(conn);
+    if (!post_data.HasMember("acks") || !post_data["acks"].IsArray())
+      return sendInvalidFormat(conn);
+    std::vector<uint64_t> ids;
+    for (auto& id : post_data["acks"].GetArray()) {
+      ids.push_back(id.GetUint64());
+    }
+    rapidjson::Document reply = rapidjson::Document(rapidjson::kObjectType);
+    reply.AddMember("acks", rapidjson::kObjectType, reply.GetAllocator());
+    for (auto& id : ids) {
+      rapidjson::Value key(std::to_string(id).c_str(), reply.GetAllocator());
+      reply["acks"].AddMember(key, std::find(indexed_events_.begin(), indexed_events_.end(), id) != indexed_events_.end() ? true : false, reply.GetAllocator());
+    }
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    reply.Accept(writer);
+
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", buffer.GetSize());
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, "%s" , buffer.GetString());
+    return true;
+  }
+
+  bool sendInvalidFormat(struct mg_connection* conn) {
+    constexpr const char * body = "{\"text\":\"Invalid data format\",\"code\":6}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  std::vector<uint64_t> indexed_events_;
+};
+
+class MockSplunkHEC {
+ public:
+  static constexpr const char* TOKEN = "Splunk 822f7d13-2b70-4f8c-848b-86edfc251222";
+
+  static inline std::vector<uint64_t> indexed_events = {0, 1};
+
+  explicit MockSplunkHEC(std::string port) : port_(std::move(port)) {
+    std::vector<std::string> options;
+    options.emplace_back("listening_ports");
+    options.emplace_back(port_);
+    server_.reset(new CivetServer(options, &callbacks_, &logger_));

Review comment:
       good idea, fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-22bed17b8af51ae7e6dba78a65e7a6c2a0e66fe55431bf0d27ee8508055afaa6R176-R193

##########
File path: extensions/splunk/SplunkHECProcessor.cpp
##########
@@ -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.
+ */
+
+#include "SplunkHECProcessor.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property SplunkHECProcessor::Hostname(core::PropertyBuilder::createProperty("Hostname")
+    ->withDescription("The ip address or hostname of the Splunk server.")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Port(core::PropertyBuilder::createProperty("Port")
+    ->withDescription("The HTTP Event Collector HTTP Port Number.")
+    ->withDefaultValue("8088")->isRequired(true)->build());

Review comment:
       :+1:  added it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-8464594c14ae65a1b12e66a4dbe2e5afa51be34981fc286fcd9aac9bb5dee1f1R30




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777461024



##########
File path: docker/test/integration/minifi/core/SplunkContainer.py
##########
@@ -0,0 +1,26 @@
+import logging
+from .Container import Container
+
+
+class SplunkContainer(Container):
+    def __init__(self, name, vols, network, image_store):
+        super().__init__(name, 'splunk', vols, network, image_store)
+
+    def get_startup_finished_log_entry(self):
+        return "Ansible playbook complete, will begin streaming splunkd_stderr.log"
+
+    def deploy(self):
+        if not self.set_deployed():
+            return
+
+        logging.info('Creating and running Splunk docker container...')
+        self.client.containers.run(
+            self.image_store.get_image(self.get_engine()),
+            detach=True,
+            name=self.name,
+            network=self.network.name,
+            environment=[
+                "SPLUNK_START_ARGS=--accept-license",

Review comment:
       They have different license terms based on the type we are using. 
   https://docs.splunk.com/Documentation/Splunk/8.2.4/Admin/TypesofSplunklicenses
   The whole eula can be found here: https://www.splunk.com/eula/sii/1.4
   
   
   I changed this section in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/b79afdea234a890818fa39c6a5ded4fb5fc117d8 so we explicity accept the free license which can be used to for these kinds of tests(we only use this container for the integration tests) more about it here: https://docs.splunk.com/Documentation/Splunk/8.2.4/Admin/TypesofSplunklicenses#Free_license
   

##########
File path: extensions/splunk/PutSplunkHTTP.h
##########
@@ -0,0 +1,54 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <memory>
+
+#include "SplunkHECProcessor.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+class PutSplunkHTTP final : public SplunkHECProcessor {

Review comment:
       fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-fffe34ee91301ca41f9e7bc23592fde7ae3205eb7e6e7c0c4e796694b5babac6R51-R54




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777581938



##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -37,6 +37,24 @@ namespace minifi {
 namespace utils {
 namespace timeutils {
 
+/**
+ * Converts the time point to the elapsed time since epoch
+ * @returns TimeUnit since epoch
+ */
+template<typename TimeUnit, typename TimePoint>
+uint64_t getTimestamp(const TimePoint& time_point) {
+  return std::chrono::duration_cast<TimeUnit>(time_point.time_since_epoch()).count();
+}
+
+/**
+ * Converts the time since epoch into a time point
+ * @returns the time point matching the input timestamp
+ */
+template<typename TimeUnit, typename ClockType>
+std::chrono::time_point<ClockType> getTimePoint(uint64_t timestamp) {
+  return std::chrono::time_point<ClockType>() + TimeUnit(timestamp);
+}

Review comment:
       [I criticized these in your other PR](https://github.com/apache/nifi-minifi-cpp/pull/1225#discussion_r767688269), which would apply here, too. If you think that they are useful, feel free to reintroduce them there, but if they don't improve the readability, you may want to consider removing them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781045981



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);

Review comment:
       as we don't sanitize the parameters they should not contain reserved characters, we should either document this, or check ourselves and throw an error (possibly through an URIComponentValidator, according to [this](https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/encodeURIComponent) non-reserved characters are `A-Z a-z 0-9 - _ . ! ~ * ' ( )`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781281354



##########
File path: bstrp_functions.sh
##########
@@ -445,6 +446,7 @@ read_feature_options(){
     x) ToggleFeature AZURE_ENABLED ;;
     y) ToggleFeature SYSTEMD_ENABLED ;;
     z) ToggleFeature NANOFI_ENABLED ;;
+    aa) ToggleFeature SPLUNK_ENABLED ;;

Review comment:
       I am not sure what would be the correct way to phrase this...
   How about this? https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/7a0401fdaf024ac39ec0b6f5cd2527c5c98762b4




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781061253



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());

Review comment:
       we could specify the type with `withDefaultValue<uint64_t>(1000)` here as well




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781066175



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {

Review comment:
       I know this is how it's done in lot of places, but I think if we specify the validator we can directly query the milliseconds with `getProperty<uint64_t>(MaximumWaitingTime)`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r782079283



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));

Review comment:
       :man_facepalming: you are absolutely right, replaced the vector with unordered_set, and added a test where this scenario is also checked
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/fe26bccb340e9ac50052111656f3c0df4006812d




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777461103



##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -37,6 +37,24 @@ namespace minifi {
 namespace utils {
 namespace timeutils {
 
+/**
+ * Converts the time point to the elapsed time since epoch
+ * @returns TimeUnit since epoch
+ */
+template<typename TimeUnit, typename TimePoint>
+uint64_t getTimeStamp(const TimePoint& time_point) {

Review comment:
       nice catch, changed this and the usages in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-4a76905d55704437ae0a7a4ee434a73f057c105c97bdf8756f5747b4fbc65e9fR45

##########
File path: extensions/splunk/SplunkHECProcessor.cpp
##########
@@ -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.
+ */
+
+#include "SplunkHECProcessor.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property SplunkHECProcessor::Hostname(core::PropertyBuilder::createProperty("Hostname")
+    ->withDescription("The ip address or hostname of the Splunk server.")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Port(core::PropertyBuilder::createProperty("Port")
+    ->withDescription("The HTTP Event Collector HTTP Port Number.")
+    ->withDefaultValue("8088")->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Token(core::PropertyBuilder::createProperty("Token")
+    ->withDescription("HTTP Event Collector token starting with the string Splunk. For example \'Splunk 1234578-abcd-1234-abcd-1234abcd\'")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::SplunkRequestChannel(core::PropertyBuilder::createProperty("Splunk Request Channel")
+    ->withDescription("Identifier of the used request channel.")->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::SSLContext(core::PropertyBuilder::createProperty("SSL Context Service")
+    ->withDescription("The SSL Context Service used to provide client certificate "
+                      "information for TLS/SSL (https) connections.")
+    ->isRequired(false)->withExclusiveProperty("Remote URL", "^http:.*$")
+    ->asType<minifi::controllers::SSLContextService>()->build());
+
+void SplunkHECProcessor::initialize() {
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel});
+}
+
+void SplunkHECProcessor::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+  if (!context->getProperty(Hostname.getName(), hostname_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Hostname");
+
+  if (!context->getProperty(Port.getName(), port_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Port");
+
+  if (!context->getProperty(Token.getName(), token_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get Token");
+
+  if (!context->getProperty(SplunkRequestChannel.getName(), request_channel_))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Failed to get SplunkRequestChannel");
+}
+
+std::string SplunkHECProcessor::getUrl() const {
+  return hostname_ + ":" + port_;

Review comment:
       changed it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/25e4878262c6ec80237339cd7388e7caab830ea0#diff-fffe34ee91301ca41f9e7bc23592fde7ae3205eb7e6e7c0c4e796694b5babac6R56




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777471185



##########
File path: docker/test/integration/minifi/core/SplunkContainer.py
##########
@@ -0,0 +1,26 @@
+import logging
+from .Container import Container
+
+
+class SplunkContainer(Container):
+    def __init__(self, name, vols, network, image_store):
+        super().__init__(name, 'splunk', vols, network, image_store)
+
+    def get_startup_finished_log_entry(self):
+        return "Ansible playbook complete, will begin streaming splunkd_stderr.log"
+
+    def deploy(self):
+        if not self.set_deployed():
+            return
+
+        logging.info('Creating and running Splunk docker container...')
+        self.client.containers.run(
+            self.image_store.get_image(self.get_engine()),
+            detach=True,
+            name=self.name,
+            network=self.network.name,
+            environment=[
+                "SPLUNK_START_ARGS=--accept-license",

Review comment:
       This is non-free software, but I think it's fine as long as we are not shipping any of it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781119396



##########
File path: extensions/splunk/SplunkHECProcessor.cpp
##########
@@ -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.
+ */
+
+#include "SplunkHECProcessor.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property SplunkHECProcessor::Hostname(core::PropertyBuilder::createProperty("Hostname")
+    ->withDescription("The ip address or hostname of the Splunk server.")
+    ->isRequired(true)->build());
+
+const core::Property SplunkHECProcessor::Port(core::PropertyBuilder::createProperty("Port")
+    ->withDescription("The HTTP Event Collector HTTP Port Number.")
+    ->withDefaultValue("8088")->isRequired(true)->build());

Review comment:
       we could specify the `PortValidator` here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781921110



##########
File path: extensions/splunk/tests/MockSplunkHEC.h
##########
@@ -0,0 +1,214 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+#include <CivetServer.h>
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+
+
+class MockSplunkHandler : public CivetHandler {
+ public:
+  explicit MockSplunkHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : token_(std::move(token)), assertions_(assertions) {
+  }
+
+  enum HeaderResult {
+    MissingAuth,
+    InvalidAuth,
+    MissingReqChannel,
+    HeadersOk
+  };
+
+  bool handlePost(CivetServer*, struct mg_connection *conn) override {
+    switch (checkHeaders(conn)) {
+      case MissingAuth:
+        return send401(conn);
+      case InvalidAuth:
+        return send403(conn);
+      case MissingReqChannel:
+        return send400(conn);
+      case HeadersOk:
+        return handlePostImpl(conn);
+    }
+    return false;
+  }
+
+  HeaderResult checkHeaders(struct mg_connection *conn) const {
+    const struct mg_request_info* req_info = mg_get_request_info(conn);
+    assertions_(req_info);
+    auto auth_header = std::find_if(std::begin(req_info->http_headers),
+                                    std::end(req_info->http_headers),
+                                    [](auto header) -> bool {return strcmp(header.name, "Authorization") == 0;});
+    if (auth_header == std::end(req_info->http_headers))
+      return MissingAuth;
+    if (strcmp(auth_header->value, token_.c_str()) != 0)
+      return InvalidAuth;
+
+    auto request_channel_header = std::find_if(std::begin(req_info->http_headers),
+                                               std::end(req_info->http_headers),
+                                               [](auto header) -> bool {return strcmp(header.name, "X-Splunk-Request-Channel") == 0;});
+
+    if (request_channel_header == std::end(req_info->http_headers))
+      return MissingReqChannel;
+    return HeadersOk;
+  }
+
+  bool send400(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Data channel is missing\",\"code\":10}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send401(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Token is required\",\"code\":2}";
+    mg_printf(conn, "HTTP/1.1 401 Unauthorized\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send403(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Invalid token\",\"code\":4}";
+    mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+ protected:
+  virtual bool handlePostImpl(struct mg_connection *conn) = 0;
+  std::string token_;
+  std::function<void(const struct mg_request_info *request_info)>& assertions_;
+};
+
+class RawCollectorHandler : public MockSplunkHandler {
+ public:
+  explicit RawCollectorHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : MockSplunkHandler(std::move(token), assertions) {}
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    constexpr const char * body = "{\"text\":\"Success\",\"code\":0,\"ackId\":808}";
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+};
+
+class AckIndexerHandler : public MockSplunkHandler {
+ public:
+  explicit AckIndexerHandler(std::string token, std::vector<uint64_t> indexed_events, std::function<void(const struct mg_request_info *request_info)>& assertions)
+      : MockSplunkHandler(std::move(token), assertions), indexed_events_(indexed_events) {}
+
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    std::vector<char> data;
+    data.reserve(2048);
+    mg_read(conn, data.data(), 2048);
+    rapidjson::Document post_data;
+
+    rapidjson::ParseResult parse_result = post_data.Parse<rapidjson::kParseStopWhenDoneFlag>(data.data());
+    if (parse_result.IsError())
+      return sendInvalidFormat(conn);
+    if (!post_data.HasMember("acks") || !post_data["acks"].IsArray())
+      return sendInvalidFormat(conn);
+    std::vector<uint64_t> ids;
+    for (auto& id : post_data["acks"].GetArray()) {
+      ids.push_back(id.GetUint64());
+    }
+    rapidjson::Document reply = rapidjson::Document(rapidjson::kObjectType);
+    reply.AddMember("acks", rapidjson::kObjectType, reply.GetAllocator());
+    for (auto& id : ids) {
+      rapidjson::Value key(std::to_string(id).c_str(), reply.GetAllocator());
+      reply["acks"].AddMember(key, std::find(indexed_events_.begin(), indexed_events_.end(), id) != indexed_events_.end() ? true : false, reply.GetAllocator());
+    }
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    reply.Accept(writer);
+
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", buffer.GetSize());
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, "%s" , buffer.GetString());
+    return true;
+  }
+
+  bool sendInvalidFormat(struct mg_connection* conn) {
+    constexpr const char * body = "{\"text\":\"Invalid data format\",\"code\":6}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  std::vector<uint64_t> indexed_events_;
+};
+
+class MockSplunkHEC {
+ public:
+  static constexpr const char* TOKEN = "Splunk 822f7d13-2b70-4f8c-848b-86edfc251222";
+
+  static inline std::vector<uint64_t> indexed_events = {0, 1};
+
+  explicit MockSplunkHEC(std::string port) : port_(std::move(port)) {
+    std::vector<std::string> options;
+    options.emplace_back("listening_ports");
+    options.emplace_back(port_);
+    server_.reset(new CivetServer(options, &callbacks_, &logger_));

Review comment:
       moving the `CivetLibrary` into its own header under `libminifi/test` might have its merits but I do not insist on it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781033337



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");

Review comment:
       nitpick: shouldn't this be "FlowFiles that failed to be sent to ..."? (I'm not really sure about this)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r779753999



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       I mean the include in all 3 cases




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777584530



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {
+  session.read(flow_file, &payload_callback);
+  payload_callback_obj.ptr = &payload_callback;
+  payload_callback_obj.pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(&payload_callback_obj);
+  client.setSeekFunction(&payload_callback_obj);
+
+  auto content_type = getContentType(context, *flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}

Review comment:
       This if statement looks a bit confusing with the multiple closing braces after it. Consider enclosing the if body in braces, or you can use optional map as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778834357



##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"
+
+using QuerySplunkIndexingStatus = org::apache::nifi::minifi::extensions::splunk::QuerySplunkIndexingStatus;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+using UpdateAttribute = org::apache::nifi::minifi::processors::UpdateAttribute;
+using namespace std::chrono_literals;  // NOLINT(build/namespaces)

Review comment:
       fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-84834c115fc3bbd69a5501ad9715f71721ee1fb6fa3b87dcdd107807ce004da2R33




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781907424



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));

Review comment:
       if we have 3 flowfiles with the same id, the first one is emplaced in the map, the second one is detected routed to failure and the id is placed in the vector, the third one is detected routed to failure and the id is placed in the vector
   iterating through the vector encounters the same id twice, and the `std::unordered_map<>::at` will throw (as we have already erased the element)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r782082669



##########
File path: extensions/http-curl/client/HTTPClient.cpp
##########
@@ -212,7 +212,10 @@ void HTTPClient::setContentType(std::string content_type) {
 }
 
 std::string HTTPClient::escape(std::string string_to_escape) {
-  return curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()));
+  char* escaped_chars = curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()));
+  std::string escaped_string(escaped_chars);
+  curl_free(escaped_chars);
+  return escaped_string;

Review comment:
       In theory string's constructor can throw which would result in leaking `escaped_chars`. Consider using `unique_ptr` with a custom curl deleter.
   ```suggestion
     struct curl_deleter { void operator()(void* p) noexcept { curl_free(p); } };
     std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
     std::string escaped_string(escaped_chars.get());
     return escaped_string;
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits closed pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
fgerlits closed pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781118596



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));

Review comment:
       should we handle if there are multiple flowfiles with the same ack_id? (currently I think we do not specify what should happen to the abandoned flowfile and the whole session will get rolled back)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780321716



##########
File path: docker/test/integration/features/splunk.feature
##########
@@ -0,0 +1,24 @@
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: A MiNiFi instance transfers data to a Splunk HEC

Review comment:
       Thanks, looks great!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778938184



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -60,6 +60,12 @@ def start_kafka_broker(self):
         self.cluster.deploy('kafka-broker')
         assert self.wait_for_container_startup_to_finish('kafka-broker')
 
+    def start_splunk(self):
+        self.cluster.acquire_container('splunk', 'splunk')
+        self.cluster.deploy('splunk')
+        assert self.wait_for_container_startup_to_finish('splunk')
+        assert self.cluster.enable_hec_indexer('splunk', 'splunk_hec_token')

Review comment:
       It could be avoided if we would keep the source file and repeatedly rerun the flow until it succeeds, but maybe this solution makes the test case more stable and reliable. I'm okay with keeping it like this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781014387



##########
File path: PROCESSORS.md
##########
@@ -50,7 +50,9 @@
 - [PutS3Object](#puts3object)
 - [PutSFTP](#putsftp)
 - [PutSQL](#putsql)
+- [PutSplunkHTTP](#putsplunkhttp)

Review comment:
       nitpick: this might not be in alphabetical order




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778837731



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));
+  }
+  return undetermined_flow_files;
+}
+
+std::string getAckIdsAsPayload(const std::unordered_map<uint64_t, FlowFileWithIndexStatus>& undetermined_flow_files) {
+  rapidjson::Document payload = rapidjson::Document(rapidjson::kObjectType);
+  payload.AddMember("acks", rapidjson::kArrayType, payload.GetAllocator());
+  for (const auto& [ack_id, ff_status] : undetermined_flow_files) {
+    payload["acks"].PushBack(ack_id, payload.GetAllocator());
+  }
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  payload.Accept(writer);
+  return buffer.GetString();
+}
+
+void getIndexingStatusFromSplunk(utils::HTTPClient& client, std::unordered_map<uint64_t, FlowFileWithIndexStatus>& undetermined_flow_files) {
+  rapidjson::Document response;
+  if (client.submit()) {

Review comment:
       good idea, I refactored this function a bit https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-a2db2ff59dd1ebf5f1e3e053781c55708df909e3009ee21015da3fe04218def4R127-R140




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778836430



##########
File path: extensions/splunk/tests/PutSplunkHTTPTests.cpp
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "PutSplunkHTTP.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "MockSplunkHEC.h"
+
+using PutSplunkHTTP = org::apache::nifi::minifi::extensions::splunk::PutSplunkHTTP;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+
+
+TEST_CASE("PutSplunkHTTP tests", "[putsplunkhttp]") {
+  MockSplunkHEC mock_splunk_hec("10133");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto put_splunk_http = std::dynamic_pointer_cast<PutSplunkHTTP>(plan->addProcessor("PutSplunkHTTP", "put_splunk_http"));
+  auto read_from_success = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_success"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, put_splunk_http);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Success, read_from_success);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Failure, read_from_failure);
+
+  read_from_success->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Hostname.getName(), "localhost");
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Happy path") {

Review comment:
       sure thing, added it in e130d7d




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778838665



##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -37,6 +37,24 @@ namespace minifi {
 namespace utils {
 namespace timeutils {
 
+/**
+ * Converts the time point to the elapsed time since epoch
+ * @returns TimeUnit since epoch
+ */
+template<typename TimeUnit, typename TimePoint>
+uint64_t getTimestamp(const TimePoint& time_point) {
+  return std::chrono::duration_cast<TimeUnit>(time_point.time_since_epoch()).count();
+}
+
+/**
+ * Converts the time since epoch into a time point
+ * @returns the time point matching the input timestamp
+ */
+template<typename TimeUnit, typename ClockType>
+std::chrono::time_point<ClockType> getTimePoint(uint64_t timestamp) {
+  return std::chrono::time_point<ClockType>() + TimeUnit(timestamp);
+}

Review comment:
       sure thing :+1: , I removed these in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-4a76905d55704437ae0a7a4ee434a73f057c105c97bdf8756f5747b4fbc65e9fL40-L57




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r780422568



##########
File path: docker/requirements.txt
##########
@@ -6,3 +6,4 @@ confluent-kafka==1.7.0
 PyYAML==5.4.1
 m2crypto==0.37.1
 watchdog==2.1.2
+pyopenssl

Review comment:
       specified it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/e77a2891da3ba880f1a24b8a74757d572dc2ccde




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778837957



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       The TimeUtil header is probably no longer needed

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"

Review comment:
       The TimeUtil header is probably no longer needed

##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"

Review comment:
       The TimeUtil header is probably no longer needed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781125574



##########
File path: extensions/splunk/tests/MockSplunkHEC.h
##########
@@ -0,0 +1,214 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+#include <CivetServer.h>
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+
+
+class MockSplunkHandler : public CivetHandler {
+ public:
+  explicit MockSplunkHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : token_(std::move(token)), assertions_(assertions) {
+  }
+
+  enum HeaderResult {
+    MissingAuth,
+    InvalidAuth,
+    MissingReqChannel,
+    HeadersOk
+  };
+
+  bool handlePost(CivetServer*, struct mg_connection *conn) override {
+    switch (checkHeaders(conn)) {
+      case MissingAuth:
+        return send401(conn);
+      case InvalidAuth:
+        return send403(conn);
+      case MissingReqChannel:
+        return send400(conn);
+      case HeadersOk:
+        return handlePostImpl(conn);
+    }
+    return false;
+  }
+
+  HeaderResult checkHeaders(struct mg_connection *conn) const {
+    const struct mg_request_info* req_info = mg_get_request_info(conn);
+    assertions_(req_info);
+    auto auth_header = std::find_if(std::begin(req_info->http_headers),
+                                    std::end(req_info->http_headers),
+                                    [](auto header) -> bool {return strcmp(header.name, "Authorization") == 0;});
+    if (auth_header == std::end(req_info->http_headers))
+      return MissingAuth;
+    if (strcmp(auth_header->value, token_.c_str()) != 0)
+      return InvalidAuth;
+
+    auto request_channel_header = std::find_if(std::begin(req_info->http_headers),
+                                               std::end(req_info->http_headers),
+                                               [](auto header) -> bool {return strcmp(header.name, "X-Splunk-Request-Channel") == 0;});
+
+    if (request_channel_header == std::end(req_info->http_headers))
+      return MissingReqChannel;
+    return HeadersOk;
+  }
+
+  bool send400(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Data channel is missing\",\"code\":10}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send401(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Token is required\",\"code\":2}";
+    mg_printf(conn, "HTTP/1.1 401 Unauthorized\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  bool send403(struct mg_connection *conn) const {
+    constexpr const char * body = "{\"text\":\"Invalid token\",\"code\":4}";
+    mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+ protected:
+  virtual bool handlePostImpl(struct mg_connection *conn) = 0;
+  std::string token_;
+  std::function<void(const struct mg_request_info *request_info)>& assertions_;
+};
+
+class RawCollectorHandler : public MockSplunkHandler {
+ public:
+  explicit RawCollectorHandler(std::string token, std::function<void(const struct mg_request_info *request_info)>& assertions) : MockSplunkHandler(std::move(token), assertions) {}
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    constexpr const char * body = "{\"text\":\"Success\",\"code\":0,\"ackId\":808}";
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+};
+
+class AckIndexerHandler : public MockSplunkHandler {
+ public:
+  explicit AckIndexerHandler(std::string token, std::vector<uint64_t> indexed_events, std::function<void(const struct mg_request_info *request_info)>& assertions)
+      : MockSplunkHandler(std::move(token), assertions), indexed_events_(indexed_events) {}
+
+ protected:
+  bool handlePostImpl(struct mg_connection* conn) override {
+    std::vector<char> data;
+    data.reserve(2048);
+    mg_read(conn, data.data(), 2048);
+    rapidjson::Document post_data;
+
+    rapidjson::ParseResult parse_result = post_data.Parse<rapidjson::kParseStopWhenDoneFlag>(data.data());
+    if (parse_result.IsError())
+      return sendInvalidFormat(conn);
+    if (!post_data.HasMember("acks") || !post_data["acks"].IsArray())
+      return sendInvalidFormat(conn);
+    std::vector<uint64_t> ids;
+    for (auto& id : post_data["acks"].GetArray()) {
+      ids.push_back(id.GetUint64());
+    }
+    rapidjson::Document reply = rapidjson::Document(rapidjson::kObjectType);
+    reply.AddMember("acks", rapidjson::kObjectType, reply.GetAllocator());
+    for (auto& id : ids) {
+      rapidjson::Value key(std::to_string(id).c_str(), reply.GetAllocator());
+      reply["acks"].AddMember(key, std::find(indexed_events_.begin(), indexed_events_.end(), id) != indexed_events_.end() ? true : false, reply.GetAllocator());
+    }
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    reply.Accept(writer);
+
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu", buffer.GetSize());
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, "%s" , buffer.GetString());
+    return true;
+  }
+
+  bool sendInvalidFormat(struct mg_connection* conn) {
+    constexpr const char * body = "{\"text\":\"Invalid data format\",\"code\":6}";
+    mg_printf(conn, "HTTP/1.1 400 Bad Request\r\n");
+    mg_printf(conn, "Content-length: %lu", strlen(body));
+    mg_printf(conn, "\r\n\r\n");
+    mg_printf(conn, body);
+    return true;
+  }
+
+  std::vector<uint64_t> indexed_events_;
+};
+
+class MockSplunkHEC {
+ public:
+  static constexpr const char* TOKEN = "Splunk 822f7d13-2b70-4f8c-848b-86edfc251222";
+
+  static inline std::vector<uint64_t> indexed_events = {0, 1};
+
+  explicit MockSplunkHEC(std::string port) : port_(std::move(port)) {
+    std::vector<std::string> options;
+    options.emplace_back("listening_ports");
+    options.emplace_back(port_);
+    server_.reset(new CivetServer(options, &callbacks_, &logger_));

Review comment:
       I think we need to initialize the civet library to safely use it like in TestServer.h




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781059919



##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());

Review comment:
       we could specify the validator with `withDefaultValue<core::TimePeriodValue>("1 hour")` and get feedback on a malformed property at yml parsing time




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r777534222



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -60,6 +60,12 @@ def start_kafka_broker(self):
         self.cluster.deploy('kafka-broker')
         assert self.wait_for_container_startup_to_finish('kafka-broker')
 
+    def start_splunk(self):
+        self.cluster.acquire_container('splunk', 'splunk')
+        self.cluster.deploy('splunk')
+        assert self.wait_for_container_startup_to_finish('splunk')
+        assert self.cluster.enable_hec_indexer('splunk', 'splunk_hec_token')

Review comment:
       From the test's point of view is it necessary to start splunk before the minifi process or is it only done separately for us to be able to enable the hec indexer? In the latter case it could be possible to have the hec indexer enabling be set as part of the entrypoint of the container (like a single command starting splunk then the hec indexer, or creating a starter script) then it wouldn't be necessary to have this container started separately from all the other cluster containers.

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {

Review comment:
       Indentation is a bit odd here

##########
File path: extensions/splunk/SplunkHECProcessor.h
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.
+ */
+
+#pragma once
+#include <string>
+#include <memory>
+
+#include "core/Processor.h"
+
+
+namespace org::apache::nifi::minifi::utils {
+class HTTPClient;
+}
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+class SplunkHECProcessor : public core::Processor {
+ public:
+  EXTENSIONAPI static const core::Property Hostname;
+  EXTENSIONAPI static const core::Property Port;
+  EXTENSIONAPI static const core::Property Token;
+  EXTENSIONAPI static const core::Property SplunkRequestChannel;
+  EXTENSIONAPI static const core::Property SSLContext;
+
+  explicit SplunkHECProcessor(const std::string& name, const utils::Identifier& uuid = {})
+      : Processor(name, uuid) {
+  }
+  ~SplunkHECProcessor() override = default;
+
+  void initialize() override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+
+  bool isSingleThreaded() const override {

Review comment:
       What's the reason for not support multithreading for this processor?

##########
File path: docker/test/integration/features/splunk.feature
##########
@@ -0,0 +1,24 @@
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: A MiNiFi instance transfers data to a Splunk HEC

Review comment:
       Would it be possible to set up an SSL connection test as well with this splunk environment?

##########
File path: extensions/splunk/QuerySplunkIndexingStatus.cpp
##########
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "QuerySplunkIndexingStatus.h"
+
+#include <unordered_map>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/writer.h"
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property QuerySplunkIndexingStatus::MaximumWaitingTime(core::PropertyBuilder::createProperty("Maximum Waiting Time")
+    ->withDescription("The maximum time the processor tries to acquire acknowledgement confirmation for an index, from the point of registration. "
+                      "After the given amount of time, the processor considers the index as not acknowledged and transfers the FlowFile to the \"unacknowledged\" relationship.")
+    ->withDefaultValue("1 hour")->isRequired(true)->build());
+
+const core::Property QuerySplunkIndexingStatus::MaxQuerySize(core::PropertyBuilder::createProperty("Maximum Query Size")
+    ->withDescription("The maximum number of acknowledgement identifiers the outgoing query contains in one batch. "
+                      "It is recommended not to set it too low in order to reduce network communication.")
+    ->withDefaultValue("1000")->isRequired(true)->build());
+
+const core::Relationship QuerySplunkIndexingStatus::Acknowledged("acknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was successful.");
+
+const core::Relationship QuerySplunkIndexingStatus::Unacknowledged("unacknowledged",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful. "
+    "This can happen when the acknowledgement did not happened within the time period set for Maximum Waiting Time. "
+    "FlowFiles with acknowledgement id unknown for the Splunk server will be transferred to this relationship after the Maximum Waiting Time is reached.");
+
+const core::Relationship QuerySplunkIndexingStatus::Undetermined("undetermined",
+    "A FlowFile is transferred to this relationship when the acknowledgement state is not determined. "
+    "FlowFiles transferred to this relationship might be penalized. "
+    "This happens when Splunk returns with HTTP 200 but with false response for the acknowledgement id in the flow file attribute.");
+
+const core::Relationship QuerySplunkIndexingStatus::Failure("failure",
+    "A FlowFile is transferred to this relationship when the acknowledgement was not successful due to errors during the communication, "
+    "or if the flowfile was missing the acknowledgement id");
+
+void QuerySplunkIndexingStatus::initialize() {
+  setSupportedRelationships({Acknowledged, Unacknowledged, Undetermined, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, MaximumWaitingTime, MaxQuerySize});
+}
+
+void QuerySplunkIndexingStatus::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  gsl_Expects(context && sessionFactory);
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+  std::string max_wait_time_str;
+  if (context->getProperty(MaximumWaitingTime.getName(), max_wait_time_str)) {
+    core::TimeUnit unit;
+    uint64_t max_wait_time;
+    if (core::Property::StringToTime(max_wait_time_str, max_wait_time, unit) && core::Property::ConvertTimeUnitToMS(max_wait_time, unit, max_wait_time)) {
+      max_age_ = std::chrono::milliseconds(max_wait_time);
+    }
+  }
+
+  context->getProperty(MaxQuerySize.getName(), batch_size_);
+}
+
+namespace {
+constexpr std::string_view getEndpoint() {
+  return "/services/collector/ack";
+}
+
+struct FlowFileWithIndexStatus {
+  explicit FlowFileWithIndexStatus(gsl::not_null<std::shared_ptr<core::FlowFile>>&& flow_file) : flow_file_(std::move(flow_file)) {}
+
+  gsl::not_null<std::shared_ptr<core::FlowFile>> flow_file_;
+  std::optional<bool> indexing_status_ = std::nullopt;
+};
+
+std::unordered_map<uint64_t, FlowFileWithIndexStatus> getUndeterminedFlowFiles(core::ProcessSession& session, size_t batch_size) {
+  std::unordered_map<uint64_t, FlowFileWithIndexStatus> undetermined_flow_files;
+  for (size_t i = 0; i < batch_size; ++i) {
+    auto flow = session.get();
+    if (flow == nullptr)
+      break;
+    std::optional<std::string> splunk_ack_id_str = flow->getAttribute(SPLUNK_ACK_ID);
+    if (!splunk_ack_id_str.has_value()) {
+      session.transfer(flow, QuerySplunkIndexingStatus::Failure);
+      continue;
+    }
+    uint64_t splunk_ack_id = std::stoull(splunk_ack_id_str.value());
+    undetermined_flow_files.emplace(std::make_pair(splunk_ack_id, gsl::not_null(std::move(flow))));
+  }
+  return undetermined_flow_files;
+}
+
+std::string getAckIdsAsPayload(const std::unordered_map<uint64_t, FlowFileWithIndexStatus>& undetermined_flow_files) {
+  rapidjson::Document payload = rapidjson::Document(rapidjson::kObjectType);
+  payload.AddMember("acks", rapidjson::kArrayType, payload.GetAllocator());
+  for (const auto& [ack_id, ff_status] : undetermined_flow_files) {
+    payload["acks"].PushBack(ack_id, payload.GetAllocator());
+  }
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  payload.Accept(writer);
+  return buffer.GetString();
+}
+
+void getIndexingStatusFromSplunk(utils::HTTPClient& client, std::unordered_map<uint64_t, FlowFileWithIndexStatus>& undetermined_flow_files) {
+  rapidjson::Document response;
+  if (client.submit()) {

Review comment:
       I would go for an early here, I just don't like long ifs :)

##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"
+
+using QuerySplunkIndexingStatus = org::apache::nifi::minifi::extensions::splunk::QuerySplunkIndexingStatus;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+using UpdateAttribute = org::apache::nifi::minifi::processors::UpdateAttribute;
+using namespace std::chrono_literals;  // NOLINT(build/namespaces)

Review comment:
       Can be replaced with `using namespace std::literals::chrono_literals` to remove the NOLINT marker

##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"
+
+using QuerySplunkIndexingStatus = org::apache::nifi::minifi::extensions::splunk::QuerySplunkIndexingStatus;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+using UpdateAttribute = org::apache::nifi::minifi::processors::UpdateAttribute;
+using namespace std::chrono_literals;  // NOLINT(build/namespaces)
+
+TEST_CASE("QuerySplunkIndexingStatus tests", "[querysplunkindexingstatus]") {
+  MockSplunkHEC mock_splunk_hec("10132");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto update_attribute = std::dynamic_pointer_cast<UpdateAttribute>(plan->addProcessor("UpdateAttribute", "update_attribute"));
+  auto query_splunk_indexing_status = std::dynamic_pointer_cast<QuerySplunkIndexingStatus>(plan->addProcessor("QuerySplunkIndexingStatus", "query_splunk_indexing_status"));
+  auto read_from_acknowledged = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_acknowledged"));
+  auto read_from_undetermined = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_undetermined"));
+  auto read_from_unacknowledged = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_unacknowledged"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, update_attribute);
+  plan->addConnection(update_attribute, UpdateAttribute ::Success, query_splunk_indexing_status);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Acknowledged, read_from_acknowledged);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Undetermined, read_from_undetermined);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Unacknowledged, read_from_unacknowledged);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Failure, read_from_failure);
+
+  read_from_acknowledged->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_undetermined->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_unacknowledged->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Hostname.getName(), "localhost");
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  auto response_timestamp = std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now()));
+  plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_RESPONSE_TIME, response_timestamp, true);
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Querying indexed id") {
+    plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_ACK_ID, std::to_string(MockSplunkHEC::indexed_events[0]), true);
+    test_controller.runSession(plan);
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 1);
+  }
+
+  SECTION("Querying not indexed id") {
+    plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_ACK_ID, "100", true);
+    query_splunk_indexing_status->setPenalizationPeriod(50ms);
+    test_controller.runSession(plan);
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 0);  // result penalized
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 0);
+
+    write_to_flow_file->setContent("");
+    plan->reset();
+    std::this_thread::sleep_for(std::chrono::milliseconds(100ms));
+    test_controller.runSession(plan);
+
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 1);
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 0);
+  }
+
+  SECTION("Querying not indexed old id") {

Review comment:
       Could we add a test for verifying the MaxQuerySize?

##########
File path: extensions/splunk/tests/PutSplunkHTTPTests.cpp
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "PutSplunkHTTP.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "MockSplunkHEC.h"
+
+using PutSplunkHTTP = org::apache::nifi::minifi::extensions::splunk::PutSplunkHTTP;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+
+
+TEST_CASE("PutSplunkHTTP tests", "[putsplunkhttp]") {
+  MockSplunkHEC mock_splunk_hec("10133");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto put_splunk_http = std::dynamic_pointer_cast<PutSplunkHTTP>(plan->addProcessor("PutSplunkHTTP", "put_splunk_http"));
+  auto read_from_success = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_success"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, put_splunk_http);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Success, read_from_success);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Failure, read_from_failure);
+
+  read_from_success->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Hostname.getName(), "localhost");
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Happy path") {

Review comment:
       Could we add some additional tests for checking non-default endpoint parameters? Also checking if the content type is set correctly with the default and a non-default value. I think with this great MockSplunkHEC we have a lot of options :)

##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {
+  session.read(flow_file, &payload_callback);
+  payload_callback_obj.ptr = &payload_callback;
+  payload_callback_obj.pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(&payload_callback_obj);
+  client.setSeekFunction(&payload_callback_obj);
+
+  auto content_type = getContentType(context, *flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}
+}  // namespace
+
+void PutSplunkHTTP::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session);
+
+  auto ff = session->get();
+  if (!ff) {
+    context->yield();
+    return;
+  }
+  auto flow_file = gsl::not_null(std::move(ff));
+
+  utils::HTTPClient client(getNetworkLocation() + getEndpoint(*context, flow_file), getSSLContextService(*context));
+  setHeaders(client);
+
+  auto payload_callback = std::make_unique<utils::ByteInputCallBack>();
+  auto payload_callback_obj = std::make_unique<utils::HTTPUploadCallback>();

Review comment:
       What's the benefit of allocating the callbacks on the heap?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778839757



##########
File path: extensions/splunk/tests/PutSplunkHTTPTests.cpp
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "PutSplunkHTTP.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "MockSplunkHEC.h"
+
+using PutSplunkHTTP = org::apache::nifi::minifi::extensions::splunk::PutSplunkHTTP;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+
+
+TEST_CASE("PutSplunkHTTP tests", "[putsplunkhttp]") {
+  MockSplunkHEC mock_splunk_hec("10133");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto put_splunk_http = std::dynamic_pointer_cast<PutSplunkHTTP>(plan->addProcessor("PutSplunkHTTP", "put_splunk_http"));
+  auto read_from_success = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_success"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, put_splunk_http);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Success, read_from_success);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Failure, read_from_failure);
+
+  read_from_success->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Hostname.getName(), "localhost");
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Happy path") {

Review comment:
       I'll add tests for the content_type aswell :eyes: 

##########
File path: extensions/splunk/tests/PutSplunkHTTPTests.cpp
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "PutSplunkHTTP.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "MockSplunkHEC.h"
+
+using PutSplunkHTTP = org::apache::nifi::minifi::extensions::splunk::PutSplunkHTTP;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+
+
+TEST_CASE("PutSplunkHTTP tests", "[putsplunkhttp]") {
+  MockSplunkHEC mock_splunk_hec("10133");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto put_splunk_http = std::dynamic_pointer_cast<PutSplunkHTTP>(plan->addProcessor("PutSplunkHTTP", "put_splunk_http"));
+  auto read_from_success = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_success"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, put_splunk_http);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Success, read_from_success);
+  plan->addConnection(put_splunk_http, PutSplunkHTTP::Failure, read_from_failure);
+
+  read_from_success->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Hostname.getName(), "localhost");
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(put_splunk_http, PutSplunkHTTP::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Happy path") {

Review comment:
       I'll add tests for the content_type aswell :eyes: :soon: 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r782077902



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);

Review comment:
       good catch, since it was already provided by our HTTPClient class I didnt think about this
   fixed it and also added tests (so it could be checked with valgrind)
   https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/fe26bccb340e9ac50052111656f3c0df4006812d




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781277627



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, SSLContext, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);

Review comment:
       Good point, I restructed these so the HTTPClient can escape these parameters. https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR84-R93




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r781280878



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/OptionalUtils.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");

Review comment:
       To be honest this was a shameless copy paste from the [nifi](https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunkHTTP.java#L133) version
   But I agree, we should fix this https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/412fa041455d782eedc52bdfb24c680ae7e56808#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR60




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778834034



##########
File path: extensions/splunk/PutSplunkHTTP.cpp
##########
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "PutSplunkHTTP.h"
+
+#include <vector>
+#include <utility>
+
+#include "SplunkAttributes.h"
+
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "client/HTTPClient.h"
+#include "utils/HTTPClient.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/document.h"
+
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+const core::Property PutSplunkHTTP::Source(core::PropertyBuilder::createProperty("Source")
+    ->withDescription("Basic field describing the source of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::SourceType(core::PropertyBuilder::createProperty("Source Type")
+    ->withDescription("Basic field describing the source type of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Host(core::PropertyBuilder::createProperty("Host")
+    ->withDescription("Basic field describing the host of the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::Index(core::PropertyBuilder::createProperty("Index")
+    ->withDescription("Identifies the index where to send the event. If unspecified, the event will use the default defined in splunk.")
+    ->supportsExpressionLanguage(true)->build());
+
+const core::Property PutSplunkHTTP::ContentType(core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("The media type of the event sent to Splunk. If not set, \"mime.type\" flow file attribute will be used. "
+                      "In case of neither of them is specified, this information will not be sent to the server.")
+    ->supportsExpressionLanguage(true)->build());
+
+
+const core::Relationship PutSplunkHTTP::Success("success", "FlowFiles that are sent successfully to the destination are sent to this relationship.");
+const core::Relationship PutSplunkHTTP::Failure("failure", "FlowFiles that failed to send to the destination are sent to this relationship.");
+
+void PutSplunkHTTP::initialize() {
+  setSupportedRelationships({Success, Failure});
+  setSupportedProperties({Hostname, Port, Token, SplunkRequestChannel, Source, SourceType, Host, Index, ContentType});
+}
+
+void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
+  SplunkHECProcessor::onSchedule(context, sessionFactory);
+}
+
+
+namespace {
+std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
+  std::optional<std::string> content_type = context.getProperty(PutSplunkHTTP::ContentType);
+  if (content_type.has_value())
+    return content_type;
+  return flow_file.getAttribute("mime.key");
+}
+
+
+std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file) {
+  std::stringstream endpoint;
+  endpoint << "/services/collector/raw";
+  std::vector<std::string> parameters;
+  std::string prop_value;
+  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
+    parameters.push_back("sourcetype=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
+    parameters.push_back("source=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
+    parameters.push_back("host=" + prop_value);
+  }
+  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
+    parameters.push_back("index=" + prop_value);
+  }
+  if (!parameters.empty()) {
+    endpoint << "?" << utils::StringUtils::join("&", parameters);
+  }
+  return endpoint.str();
+}
+
+bool addAttributesFromClientResponse(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  rapidjson::Document response_json;
+  rapidjson::ParseResult parse_result = response_json.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  bool result = true;
+  if (parse_result.IsError())
+    return false;
+
+  if (response_json.HasMember("code") && response_json["code"].IsInt())
+    flow_file.addAttribute(SPLUNK_RESPONSE_CODE, std::to_string(response_json["code"].GetInt()));
+  else
+    result = false;
+
+  if (response_json.HasMember("ackId") && response_json["ackId"].IsUint64())
+    flow_file.addAttribute(SPLUNK_ACK_ID, std::to_string(response_json["ackId"].GetUint64()));
+  else
+    result = false;
+
+  return result;
+}
+
+bool enrichFlowFileWithAttributes(core::FlowFile& flow_file, utils::HTTPClient& client) {
+  flow_file.addAttribute(SPLUNK_STATUS_CODE, std::to_string(client.getResponseCode()));
+  flow_file.addAttribute(SPLUNK_RESPONSE_TIME, std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now())));
+
+  return addAttributesFromClientResponse(flow_file, client) && client.getResponseCode() == 200;
+}
+
+void setFlowFileAsPayload(core::ProcessSession& session,
+                                         core::ProcessContext& context,
+                                         utils::HTTPClient& client,
+                                         const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file,
+                                         utils::ByteInputCallBack& payload_callback,
+                                         utils::HTTPUploadCallback& payload_callback_obj) {
+  session.read(flow_file, &payload_callback);
+  payload_callback_obj.ptr = &payload_callback;
+  payload_callback_obj.pos = 0;
+  client.appendHeader("Content-Length", std::to_string(flow_file->getSize()));
+
+  client.setUploadCallback(&payload_callback_obj);
+  client.setSeekFunction(&payload_callback_obj);
+
+  auto content_type = getContentType(context, *flow_file);
+  if (content_type.has_value())
+    client.setContentType(content_type.value());
+}
+}  // namespace
+
+void PutSplunkHTTP::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session);
+
+  auto ff = session->get();
+  if (!ff) {
+    context->yield();
+    return;
+  }
+  auto flow_file = gsl::not_null(std::move(ff));
+
+  utils::HTTPClient client(getNetworkLocation() + getEndpoint(*context, flow_file), getSSLContextService(*context));
+  setHeaders(client);
+
+  auto payload_callback = std::make_unique<utils::ByteInputCallBack>();
+  auto payload_callback_obj = std::make_unique<utils::HTTPUploadCallback>();

Review comment:
       :+1: nothing, it was due to copy paste from InvokeHTTP...
   fixed in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/a0e996b4844c92dd3d2cc4d70955ad4ea5697f31#diff-2633ef573b024e894869a6a974a55671c3468db3eff99e4cdc646a081a700efdR162-R163




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778836242



##########
File path: extensions/splunk/tests/QuerySplunkIndexingStatusTests.cpp
##########
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+
+#include "QuerySplunkIndexingStatus.h"
+#include "MockSplunkHEC.h"
+#include "SplunkAttributes.h"
+#include "TestBase.h"
+#include "processors/UpdateAttribute.h"
+#include "ReadFromFlowFileTestProcessor.h"
+#include "WriteToFlowFileTestProcessor.h"
+#include "utils/TimeUtil.h"
+
+using QuerySplunkIndexingStatus = org::apache::nifi::minifi::extensions::splunk::QuerySplunkIndexingStatus;
+using ReadFromFlowFileTestProcessor = org::apache::nifi::minifi::processors::ReadFromFlowFileTestProcessor;
+using WriteToFlowFileTestProcessor = org::apache::nifi::minifi::processors::WriteToFlowFileTestProcessor;
+using UpdateAttribute = org::apache::nifi::minifi::processors::UpdateAttribute;
+using namespace std::chrono_literals;  // NOLINT(build/namespaces)
+
+TEST_CASE("QuerySplunkIndexingStatus tests", "[querysplunkindexingstatus]") {
+  MockSplunkHEC mock_splunk_hec("10132");
+
+  TestController test_controller;
+  auto plan = test_controller.createPlan();
+  auto write_to_flow_file = std::dynamic_pointer_cast<WriteToFlowFileTestProcessor>(plan->addProcessor("WriteToFlowFileTestProcessor", "write_to_flow_file"));
+  auto update_attribute = std::dynamic_pointer_cast<UpdateAttribute>(plan->addProcessor("UpdateAttribute", "update_attribute"));
+  auto query_splunk_indexing_status = std::dynamic_pointer_cast<QuerySplunkIndexingStatus>(plan->addProcessor("QuerySplunkIndexingStatus", "query_splunk_indexing_status"));
+  auto read_from_acknowledged = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_acknowledged"));
+  auto read_from_undetermined = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_undetermined"));
+  auto read_from_unacknowledged = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_unacknowledged"));
+  auto read_from_failure = std::dynamic_pointer_cast<ReadFromFlowFileTestProcessor>(plan->addProcessor("ReadFromFlowFileTestProcessor", "read_from_failure"));
+
+  plan->addConnection(write_to_flow_file, WriteToFlowFileTestProcessor::Success, update_attribute);
+  plan->addConnection(update_attribute, UpdateAttribute ::Success, query_splunk_indexing_status);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Acknowledged, read_from_acknowledged);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Undetermined, read_from_undetermined);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Unacknowledged, read_from_unacknowledged);
+  plan->addConnection(query_splunk_indexing_status, QuerySplunkIndexingStatus::Failure, read_from_failure);
+
+  read_from_acknowledged->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_undetermined->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_unacknowledged->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+  read_from_failure->setAutoTerminatedRelationships({ReadFromFlowFileTestProcessor::Success});
+
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Hostname.getName(), "localhost");
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Port.getName(), mock_splunk_hec.getPort());
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::Token.getName(), MockSplunkHEC::TOKEN);
+  plan->setProperty(query_splunk_indexing_status, QuerySplunkIndexingStatus::SplunkRequestChannel.getName(), "a12254b4-f481-435d-896d-3b6033eabe58");
+
+  auto response_timestamp = std::to_string(utils::timeutils::getTimestamp<std::chrono::milliseconds>(std::chrono::system_clock::now()));
+  plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_RESPONSE_TIME, response_timestamp, true);
+
+  write_to_flow_file->setContent("foobar");
+
+  SECTION("Querying indexed id") {
+    plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_ACK_ID, std::to_string(MockSplunkHEC::indexed_events[0]), true);
+    test_controller.runSession(plan);
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 1);
+  }
+
+  SECTION("Querying not indexed id") {
+    plan->setProperty(update_attribute, org::apache::nifi::minifi::extensions::splunk::SPLUNK_ACK_ID, "100", true);
+    query_splunk_indexing_status->setPenalizationPeriod(50ms);
+    test_controller.runSession(plan);
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 0);  // result penalized
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 0);
+
+    write_to_flow_file->setContent("");
+    plan->reset();
+    std::this_thread::sleep_for(std::chrono::milliseconds(100ms));
+    test_controller.runSession(plan);
+
+    CHECK(read_from_failure->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_undetermined->numberOfFlowFilesRead() == 1);
+    CHECK(read_from_unacknowledged->numberOfFlowFilesRead() == 0);
+    CHECK(read_from_acknowledged->numberOfFlowFilesRead() == 0);
+  }
+
+  SECTION("Querying not indexed old id") {

Review comment:
       sure thing, added it in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/e130d7dcf4f90e19cd36c46a5742a1f123d18eb7




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #1219: MINIFICPP-1691: PutSplunkHTTP and QuerySplunkIndexingStatus

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #1219:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1219#discussion_r778835858



##########
File path: extensions/splunk/SplunkHECProcessor.h
##########
@@ -0,0 +1,65 @@
+/**
+ * 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.
+ */
+
+#pragma once
+#include <string>
+#include <memory>
+
+#include "core/Processor.h"
+
+
+namespace org::apache::nifi::minifi::utils {
+class HTTPClient;
+}
+
+namespace org::apache::nifi::minifi::extensions::splunk {
+
+class SplunkHECProcessor : public core::Processor {
+ public:
+  EXTENSIONAPI static const core::Property Hostname;
+  EXTENSIONAPI static const core::Property Port;
+  EXTENSIONAPI static const core::Property Token;
+  EXTENSIONAPI static const core::Property SplunkRequestChannel;
+  EXTENSIONAPI static const core::Property SSLContext;
+
+  explicit SplunkHECProcessor(const std::string& name, const utils::Identifier& uuid = {})
+      : Processor(name, uuid) {
+  }
+  ~SplunkHECProcessor() override = default;
+
+  void initialize() override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+
+  bool isSingleThreaded() const override {

Review comment:
       valid point, I think it can be enabled in PutSplunkHTTP (QuerySplunkIndexingStatus should remain single threaded here the user can increase the MaxQuerySize if needed)
   
   enabled in https://github.com/apache/nifi-minifi-cpp/pull/1219/commits/3283a3aa2a51c9c250a83f1128aa18631960f688




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org