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 2022/01/03 16:12:27 UTC

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

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