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/06/13 15:46:28 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request, #1349: MINIFICPP-1843 Implement PutElasticsearchJson

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

   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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908573134


##########
extensions/elasticsearch/tests/MockElastic.h:
##########
@@ -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.
+ */
+
+#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"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+class MockElasticAuthHandler : public CivetAuthHandler {
+ public:
+  static constexpr const char* API_KEY = "VnVhQ2ZHY0JDZGJrUW0tZTVhT3g6dWkybHAyYXhUTm1zeWFrdzl0dk5udw";
+  static constexpr const char* USERNAME = "elastic";
+  static constexpr const char* PASSWORD = "elastic_password";
+
+ private:
+  bool authorize(CivetServer*, struct mg_connection* conn) override {
+    const char* authHeader = mg_get_header(conn, "Authorization");
+    if (authHeader == nullptr) {
+      return false;
+    }
+    if (strcmp(authHeader, "Basic ZWxhc3RpYzplbGFzdGljX3Bhc3N3b3Jk") == 0)
+      return true;
+    if (strcmp(authHeader, "ApiKey VnVhQ2ZHY0JDZGJrUW0tZTVhT3g6dWkybHAyYXhUTm1zeWFrdzl0dk5udw") == 0)
+      return true;
+    return false;
+  };
+};
+
+class BulkElasticHandler : public CivetHandler {
+ public:
+  void returnErrors(bool ret_errors) {
+    ret_error_ = ret_errors;
+  }
+
+ private:
+  rapidjson::Value addIndexSuccess(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    operation.AddMember("result", "created", alloc);
+    item.AddMember("index", operation, alloc);
+    return item;
+  }
+
+  rapidjson::Value addUpdateSuccess(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    operation.AddMember("result", "updated", alloc);
+    item.AddMember("update", operation, alloc);
+    return item;
+  }
+
+  rapidjson::Value addUpdateError(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    rapidjson::Value error{rapidjson::kObjectType};
+    error.AddMember("type", "document_missing_exception", alloc);
+    error.AddMember("reason", "[6]: document missing", alloc);
+    error.AddMember("index_uuid", "aAsFqTI0Tc2W0LCWgPNrOA", alloc);
+    error.AddMember("shard", "0", alloc);
+    error.AddMember("index", "index", alloc);
+    operation.AddMember("error", error, alloc);
+    item.AddMember("update", operation, alloc);
+    return item;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    char request[2048];
+    size_t chars_read = mg_read(conn, request, 2048);
+
+    std::vector<std::string> lines = utils::StringUtils::splitRemovingEmpty({request, chars_read}, "\n");
+    rapidjson::Document response{rapidjson::kObjectType};
+    response.AddMember("took", 30, response.GetAllocator());
+    response.AddMember("errors", ret_error_, response.GetAllocator());
+    response.AddMember("items", rapidjson::kArrayType, response.GetAllocator());
+    auto& items = response["items"];
+    for (const auto& line : lines) {
+      rapidjson::Document line_json;
+      line_json.Parse<rapidjson::kParseStopWhenDoneFlag>(line.data());
+      if (!line_json.HasMember("index") && !line_json.HasMember("create") && !line_json.HasMember("update") && !line_json.HasMember("delete"))
+        continue;
+
+
+      rapidjson::Value item{rapidjson::kObjectType};
+      rapidjson::Value operation{rapidjson::kObjectType};
+
+      if (ret_error_) {
+        items.PushBack(addUpdateError(response.GetAllocator()), response.GetAllocator());
+      } else {
+        if (line_json.HasMember("update"))
+          items.PushBack(addUpdateSuccess(response.GetAllocator()), response.GetAllocator());
+        else
+          items.PushBack(addIndexSuccess(response.GetAllocator()), response.GetAllocator());
+      }
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    response.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 ret_error_ = false;
+};
+
+class MockElastic {
+  struct CivetLibrary{

Review Comment:
   makes sense, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/f260dc059908a52310b14d6dbd09c9dbbe81fab5#diff-ae2a7d7ad1f15725f1e8101ea1a3c3d437942603e31e9ab66802a76b1641cad4R1



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r896848490


##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci

Review Comment:
   good idea, https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/a29b2497fecb37bae02bbe6f28d09f872e2ef279



##########
CMakeLists.txt:
##########
@@ -116,7 +116,9 @@ option(ENABLE_OPENWSMAN "Enables the Openwsman extensions." OFF)
 option(ENABLE_AZURE "Enables Azure support." ON)
 option(ENABLE_ENCRYPT_CONFIG "Enables build of encrypt-config binary." ON)
 option(ENABLE_SPLUNK "Enable Splunk support" ON)
+option(ENABLE_ELASTICSEARCH "Enable Elasticsearch support" OFF)
 option(ENABLE_GCP "Enable Google Cloud support" ON)
+option(ENABLE_PROCFS "Enable ProcFs support" OFF)

Review Comment:
   nice catch, no idea how this happened https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/500deb5ecf704d166196ef141ec2ce5791246bfc



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908213800


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");

Review Comment:
   I don't think that in its current form it is safe to reuse a HTTPClient, for example the response headers are appended to the same vector on each submit, the request header if not empty (and it is not because we set the content-type) gets a `"Expect:"` appended on each submit



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r901568353


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();
+    return;
+  }
+
+
+  client_.setPostFields(payload.str());
+  auto result = submitRequest(client_, flowfiles_in_payload.size());
+  if (!result) {
+    logger_->log_error(result.error().c_str());
+    for (const auto& flow_file_in_payload: flowfiles_in_payload)
+      session->transfer(flow_file_in_payload, Failure);
+    return;
+  }
+
+  auto& items = result->operator[]("items");
+  gsl_Expects(items.Size() == flowfiles_in_payload.size());
+  for (size_t i = 0; i < items.Size(); ++i) {
+    for (auto it = items[i].MemberBegin(); it != items[i].MemberEnd(); ++it) {
+      addAttributesFromResponse("elasticsearch", it, *flowfiles_in_payload[i]);
+    }
+    if (items[i].MemberBegin()->value.HasMember("error"))

Review Comment:
   Yeah the response json will be flattened and added as attributes.
   The error json seems quite informative to me, so I think its a good idea to include it. e.g.
   ```
   "update": {
     "_index": "index1",
     "_id": "5",
     "status": 404,
     "error": {
       "type": "document_missing_exception",
       "reason": "[5]: document missing",
       "index_uuid": "aAsFqTI0Tc2W0LCWgPNrOA",
       "shard": "0",
       "index": "index1"
     }
   }
   ```
   
   I've added a test for error relationship where this behavior can be checked out.
   https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-da99b94efb790428cd1298692800cab09ee2b398538b67ec7f0d5e4bfae2605dR91-R94



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,102 @@
+/**
+ * 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 "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");
+  minifi::test::SingleProcessorTestController test_controller{put_elasticsearch_json};
+  auto elasticsearch_credentials_controller_service = test_controller.plan->addController("ElasticsearchCredentialsControllerService", "elasticsearch_credentials_controller_service");
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                     PostElasticsearch::ElasticCredentials.getName(),
+                                     "elasticsearch_credentials_controller_service"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Hosts.getName(),
+                                    "localhost:10433"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Action.getName(),
+                                    "${elastic_action}"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Index.getName(),
+                                    "test_index"));
+
+  SECTION("Index with valid basic authentication") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Username.getName(),
+                                            MockElasticAuthHandler::USERNAME));
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Password.getName(),
+                                            MockElasticAuthHandler::PASSWORD));
+
+    std::vector<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> inputs;
+
+    auto results = test_controller.trigger({std::make_tuple<const std::string_view, std::unordered_map<std::string, std::string>>(R"({"field1":"value1"}")", {{"elastic_action", "index"}}),
+                                            std::make_tuple<const std::string_view, std::unordered_map<std::string, std::string>>(R"({"field1":"value2"}")", {{"elastic_action", "index"}})});
+    REQUIRE(results[PostElasticsearch::Success].size() == 2);
+    for (const auto& result : results[PostElasticsearch::Success]) {
+      auto attributes = result->getAttributes();
+      CHECK(attributes.contains("elasticsearch.index._id"));
+      CHECK(attributes.contains("elasticsearch.index._index"));
+    }
+  }
+
+  SECTION("Update with valid ApiKey") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::ApiKey.getName(),
+                                            MockElasticAuthHandler::API_KEY));
+    CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                            PostElasticsearch::Identifier.getName(),
+                                            "${filename}"));
+
+    auto results = test_controller.trigger(R"({"field1":"value1"}")", {{"elastic_action", "upsert"}});
+    REQUIRE(results[PostElasticsearch::Success].size() == 1);
+    auto attributes = results[PostElasticsearch::Success][0]->getAttributes();
+    CHECK(attributes.contains("elasticsearch.update._id"));
+    CHECK(attributes.contains("elasticsearch.update._index"));
+  }
+
+  SECTION("Invalid ApiKey") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::ApiKey.getName(),
+                                            "invalid_api_key"));
+
+    auto results = test_controller.trigger(R"({"field1":"value1"}")", {{"elastic_action", "create"}});
+    CHECK(results[PostElasticsearch::Failure].size() == 1);
+  }
+
+  SECTION("Invalid basic authentication") {

Review Comment:
   good idea, I've added it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-da99b94efb790428cd1298692800cab09ee2b398538b67ec7f0d5e4bfae2605dR80-R96



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {

Review Comment:
   good idea, i've refactored this part in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();
+    return;
+  }
+
+
+  client_.setPostFields(payload.str());
+  auto result = submitRequest(client_, flowfiles_in_payload.size());
+  if (!result) {
+    logger_->log_error(result.error().c_str());
+    for (const auto& flow_file_in_payload: flowfiles_in_payload)
+      session->transfer(flow_file_in_payload, Failure);
+    return;
+  }
+
+  auto& items = result->operator[]("items");

Review Comment:
   refactored this part in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }

Review Comment:
   It shouldnt happen but a log message is probably a good idea.
   https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR251



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;

Review Comment:
   makes sense, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR294



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908224869


##########
extensions/elasticsearch/tests/MockElastic.h:
##########
@@ -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.
+ */
+
+#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"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+class MockElasticAuthHandler : public CivetAuthHandler {
+ public:
+  static constexpr const char* API_KEY = "VnVhQ2ZHY0JDZGJrUW0tZTVhT3g6dWkybHAyYXhUTm1zeWFrdzl0dk5udw";
+  static constexpr const char* USERNAME = "elastic";
+  static constexpr const char* PASSWORD = "elastic_password";
+
+ private:
+  bool authorize(CivetServer*, struct mg_connection* conn) override {
+    const char* authHeader = mg_get_header(conn, "Authorization");
+    if (authHeader == nullptr) {
+      return false;
+    }
+    if (strcmp(authHeader, "Basic ZWxhc3RpYzplbGFzdGljX3Bhc3N3b3Jk") == 0)
+      return true;
+    if (strcmp(authHeader, "ApiKey VnVhQ2ZHY0JDZGJrUW0tZTVhT3g6dWkybHAyYXhUTm1zeWFrdzl0dk5udw") == 0)
+      return true;
+    return false;
+  };
+};
+
+class BulkElasticHandler : public CivetHandler {
+ public:
+  void returnErrors(bool ret_errors) {
+    ret_error_ = ret_errors;
+  }
+
+ private:
+  rapidjson::Value addIndexSuccess(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    operation.AddMember("result", "created", alloc);
+    item.AddMember("index", operation, alloc);
+    return item;
+  }
+
+  rapidjson::Value addUpdateSuccess(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    operation.AddMember("result", "updated", alloc);
+    item.AddMember("update", operation, alloc);
+    return item;
+  }
+
+  rapidjson::Value addUpdateError(rapidjson::Document::AllocatorType& alloc) {
+    rapidjson::Value item{rapidjson::kObjectType};
+    rapidjson::Value operation{rapidjson::kObjectType};
+    operation.AddMember("_index", "test", alloc);
+    operation.AddMember("_id", "1", alloc);
+    rapidjson::Value error{rapidjson::kObjectType};
+    error.AddMember("type", "document_missing_exception", alloc);
+    error.AddMember("reason", "[6]: document missing", alloc);
+    error.AddMember("index_uuid", "aAsFqTI0Tc2W0LCWgPNrOA", alloc);
+    error.AddMember("shard", "0", alloc);
+    error.AddMember("index", "index", alloc);
+    operation.AddMember("error", error, alloc);
+    item.AddMember("update", operation, alloc);
+    return item;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    char request[2048];
+    size_t chars_read = mg_read(conn, request, 2048);
+
+    std::vector<std::string> lines = utils::StringUtils::splitRemovingEmpty({request, chars_read}, "\n");
+    rapidjson::Document response{rapidjson::kObjectType};
+    response.AddMember("took", 30, response.GetAllocator());
+    response.AddMember("errors", ret_error_, response.GetAllocator());
+    response.AddMember("items", rapidjson::kArrayType, response.GetAllocator());
+    auto& items = response["items"];
+    for (const auto& line : lines) {
+      rapidjson::Document line_json;
+      line_json.Parse<rapidjson::kParseStopWhenDoneFlag>(line.data());
+      if (!line_json.HasMember("index") && !line_json.HasMember("create") && !line_json.HasMember("update") && !line_json.HasMember("delete"))
+        continue;
+
+
+      rapidjson::Value item{rapidjson::kObjectType};
+      rapidjson::Value operation{rapidjson::kObjectType};
+
+      if (ret_error_) {
+        items.PushBack(addUpdateError(response.GetAllocator()), response.GetAllocator());
+      } else {
+        if (line_json.HasMember("update"))
+          items.PushBack(addUpdateSuccess(response.GetAllocator()), response.GetAllocator());
+        else
+          items.PushBack(addIndexSuccess(response.GetAllocator()), response.GetAllocator());
+      }
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    response.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 ret_error_ = false;
+};
+
+class MockElastic {
+  struct CivetLibrary{

Review Comment:
   this civet initializer is already defined in `TestServer.h` and `MockSplunkHEC.h`, it might be time to move it to a common place



-- 
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 pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
szaszm commented on PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#issuecomment-1154103261

   Can this work with the OpenSearch fork?


-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907637828


##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,93 @@
+# 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.
+
+@no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: MiNiFi instance indexes a document on Elasticsearch using Basic Authentication
+    Given an Elasticsearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Elasticsearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Elasticsearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  Scenario: MiNiFi instance creates a document on Elasticsearch using API Key authentication

Review Comment:
   makes sense, I've remade them into Scenario Outlines in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fa725ffb8fc830c7cf8518f40334d7ee9fa0df13#diff-0256a991bbecdae5406fe625e6bca90bfbfe4c3b6390f8749b63658534c80779R40-R43



##########
docker/test/integration/features/opensearch.feature:
##########
@@ -0,0 +1,97 @@
+# 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.
+
+@no-ci  # Opensearch container requires more RAM than what the CI environment has
+Feature: PostElasticsearch works on Opensearch (Opensearch doesnt support API Keys)
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: MiNiFi instance indexes a document on Opensearch using Basic Authentication
+    Given an Opensearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Hosts" property of the PostElasticsearch processor is set to "https://opensearch:9200"
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Opensearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Opensearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  Scenario: MiNiFi instance creates a document on Opensearch using Basic Authentication

Review Comment:
   makes sense, I've remade them into Scenario Outlines in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fa725ffb8fc830c7cf8518f40334d7ee9fa0df13#diff-9f20f0de056cfd00e2e8fa2e58c861a9f213f9c15f0f2bbcb5313feb0862c500R41-R44



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908572422


##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -20,6 +20,7 @@
 #include <set>
 #include <string>
 #include <string_view>
+#include <tuple>

Review Comment:
   good catch, removed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/f260dc059908a52310b14d6dbd09c9dbbe81fab5#diff-a019248b6c700d2a0bc69681c6ea011e5afe07861a32a021f992375eeda11847L23



-- 
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 commented on a diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908284188


##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -20,6 +20,7 @@
 #include <set>
 #include <string>
 #include <string_view>
+#include <tuple>

Review Comment:
   is this include still needed?



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();

Review Comment:
   I would mark `Action`, `Hosts` and `Index` as `isRequired(true)`.



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

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 pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
szaszm commented on PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#issuecomment-1155232515

   If it works, we should advertise it IMO, since OpenSearch is actually open-source (Apache 2.0), versus Elasticsearch which is proprietary with public source (some consider it strong copyleft, very business-hostile).


-- 
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 closed pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1349: MINIFICPP-1843 Implement PostElasticsearch
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349


-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907634157


##########
bstrp_functions.sh:
##########
@@ -386,6 +386,7 @@ show_supported_features() {
   echo "AB. Kubernetes Support .........$(print_feature_status KUBERNETES_ENABLED)"
   echo "AC. Google Cloud Support .......$(print_feature_status GCP_ENABLED)"
   echo "AD. ProcFs Support .............$(print_feature_status PROCFS_ENABLED)"
+  echo "AE. Elasticsearch Support ......$(print_feature_status ELASTIC_ENABLED)"

Review Comment:
   good catch, fixed in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fa725ffb8fc830c7cf8518f40334d7ee9fa0df13#diff-90d97f08a2155d61f2b5a04dd055c4bd0e047ea49e2078994da3b9ef338bfdfbR412-R467



-- 
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 commented on a diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908139962


##########
docker/DockerVerify.sh:
##########
@@ -67,7 +67,7 @@ TEST_DIRECTORY="${docker_dir}/test/integration"
 export TEST_DIRECTORY
 
 # Add --no-logcapture to see logs interleaved with the test output
-BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers)
+BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers --tags ~@no-ci)

Review Comment:
   Yes, that's fine.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908201354


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static nonstd::expected<ElasticPayload, std::string> parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+nonstd::expected<rapidjson::Document, std::string> submitRequest(utils::HTTPClient& client, std::string&& payload, const size_t expected_items) {
+  client.setPostFields(std::move(payload));
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  } else if (object->value.IsDouble()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetDouble()));
+  } else {
+    core::logging::LoggerFactory<PostElasticsearch>::getLogger()->log_error("Unexpected %s in response json", object->value.GetType());
+  }
+}
+
+void processResponseFromElastic(const rapidjson::Document& response, core::ProcessSession& session, const std::vector<std::shared_ptr<core::FlowFile>>& flowfiles_sent) {
+  gsl_Expects(response.HasMember("items"));
+  auto& items = response["items"];
+  gsl_Expects(items.Size() == flowfiles_sent.size());
+  for (size_t i = 0; i < items.Size(); ++i) {
+    for (auto it = items[i].MemberBegin(); it != items[i].MemberEnd(); ++it) {

Review Comment:
   should we assert that `items[i]` is an object?



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r906742608


##########
docker/test/integration/resources/elasticsearch/Dockerfile:
##########
@@ -0,0 +1,7 @@
+FROM elasticsearch:8.2.2
+COPY elasticsearch.yml /usr/share/elasticsearch/config/elasticsearch.yml
+COPY certs/elastic_cert.key /usr/share/elasticsearch/config/certs/elastic_cert.key
+COPY certs/elastic_cert.pem /usr/share/elasticsearch/config/certs/elastic_cert.pem
+COPY certs/elastic_transport.key /usr/share/elasticsearch/config/certs/elastic_transport.key
+COPY certs/elastic_transport.pem /usr/share/elasticsearch/config/certs/elastic_transport.pem

Review Comment:
   Both the cert and the private keys are in PEM format. Maybe using the .key.pem and .crt.pem, or .key and .crt extension would be less confusing.



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");
+  minifi::test::SingleProcessorTestController test_controller{put_elasticsearch_json};
+  auto elasticsearch_credentials_controller_service = test_controller.plan->addController("ElasticsearchCredentialsControllerService", "elasticsearch_credentials_controller_service");
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                     PostElasticsearch::ElasticCredentials.getName(),
+                                     "elasticsearch_credentials_controller_service"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Hosts.getName(),
+                                    "localhost:10433"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Action.getName(),
+                                    "${elastic_action}"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Index.getName(),
+                                    "test_index"));
+
+  SECTION("Index with valid basic authentication") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Username.getName(),
+                                            MockElasticAuthHandler::USERNAME));
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Password.getName(),
+                                            MockElasticAuthHandler::PASSWORD));
+
+    std::vector<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> inputs;

Review Comment:
   `inputs` looks unused to me.



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");

Review Comment:
   [ES.11: Use auto to avoid redundant repetition of type names](https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#es11-use-auto-to-avoid-redundant-repetition-of-type-names)



##########
PROCESSORS.md:
##########
@@ -1640,6 +1641,35 @@ In the list below, the names of required properties appear in bold. Any other pr
 | success | All files are routed to success |
 
 
+## PostElasticsearch
+
+### Description
+
+An Elasticsearch/Opensearch post processor that uses the Elasticsearch/Opensearch _bulk REST API.
+### Properties
+
+In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
+
+| Name                                           | Default Value | Allowable Values | Description                                                                                                                                                                                                                                                                                               |
+|------------------------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Action                                         |               |                  | The type of the operation used to index (create, delete, index, update, upsert)<br/>**Supports Expression Language: true**                                                                                                                                                                                |
+| Max Batch Size                                 | 100           |                  | The maximum number of Syslog events to process at a time.                                                                                                                                                                                                                                                 |

Review Comment:
   syslog events?
   ```suggestion
   | Max Batch Size                                 | 100           |                  | The maximum number of flow files to process at a time.                                                                                                                                                                                                                                                 |
   ```



##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -297,6 +301,8 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   std::chrono::milliseconds keep_alive_idle_{-1};
 
+  std::optional<std::pair<std::string, std::string>> username_password_;

Review Comment:
   Use a proper named type over a tuple. Something like `struct BasicAuthCredentials { std::string username; std::string password; };`



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, std::string&& payload, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {

Review Comment:
   That's the purpose of using trailing return type here? Usually I prefer leading return type, except if the return type expression needs to reference the parameters (or it's easier to express it that way).



##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -57,6 +58,14 @@ class SingleProcessorTestController : public TestController {
     return trigger();
   }
 
+  auto trigger(std::initializer_list<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> flow_files) {

Review Comment:
   Use a proper container instead, like `std:array` or `std::vector`.
   
   Use a dedicated type for flow file data representation over a tuple. Could be a dead simple aggregate struct, just give it a name.



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")

Review Comment:
   syslog again, but I think this is more general



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r899133277


##########
.github/workflows/ci.yml:
##########
@@ -166,7 +166,7 @@ jobs:
           cmake -DUSE_SHARED_LIBS=ON -DCMAKE_BUILD_TYPE=Release -DSTRICT_GSL_CHECKS=AUDIT -DFAIL_ON_WARNINGS=ON -DENABLE_AWS=ON -DENABLE_AZURE=ON -DENABLE_BUSTACHE=ON -DENABLE_COAP=ON \
               -DENABLE_ENCRYPT_CONFIG=ON -DENABLE_GPS=ON -DENABLE_JNI=ON -DENABLE_LIBRDKAFKA=ON -DENABLE_LINTER=ON -DENABLE_MQTT=ON -DENABLE_NANOFI=ON -DENABLE_OPC=ON -DENABLE_OPENCV=ON \
               -DENABLE_OPENWSMAN=ON -DENABLE_OPS=ON -DENABLE_PCAP=ON -DENABLE_PYTHON=ON -DENABLE_SENSORS=ON -DENABLE_SFTP=ON -DENABLE_SQL=ON -DENABLE_SYSTEMD=ON -DENABLE_TENSORFLOW=OFF \
-              -DENABLE_USB_CAMERA=ON -DENABLE_SCRIPTING=ON -DENABLE_LUA_SCRIPTING=ON -DENABLE_KUBERNETES=ON -DENABLE_GCP=ON -DENABLE_PROCFS=ON -DCMAKE_EXPORT_COMPILE_COMMANDS=ON ..
+              -DENABLE_USB_CAMERA=ON -DENABLE_SCRIPTING=ON -DENABLE_LUA_SCRIPTING=ON -DENABLE_KUBERNETES=ON -DENABLE_GCP=ON -DENABLE_PROCFS=ON -DENABLE_ELASTICSEARCH=ON -DCMAKE_EXPORT_COMPILE_COMMANDS=ON ..

Review Comment:
   I think we should also add it to the GCC build as we usually release the GCC builds. Also should this be built with Windows as well in CI?



##########
docker/test/integration/resources/elasticsearch/Dockerfile:
##########
@@ -0,0 +1,7 @@
+FROM elasticsearch:8.2.2
+ADD elasticsearch.yml /usr/share/elasticsearch/config/elasticsearch.yml

Review Comment:
   It's usually recommended to use COPY command instead of ADD: https://github.com/hadolint/hadolint/wiki/DL3020
   Same for opensearch dockerfile



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();
+    return;
+  }
+
+
+  client_.setPostFields(payload.str());
+  auto result = submitRequest(client_, flowfiles_in_payload.size());
+  if (!result) {
+    logger_->log_error(result.error().c_str());
+    for (const auto& flow_file_in_payload: flowfiles_in_payload)
+      session->transfer(flow_file_in_payload, Failure);
+    return;
+  }
+
+  auto& items = result->operator[]("items");

Review Comment:
   Could you use `result["items"]` here make it simpler? Also is "items" always available? If so maybe an assertion could be added, or if not a check.



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;

Review Comment:
   Maybe that's just my understanding, but it seems to me `flowfiles_with_payload` may be a better name.



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has

Review Comment:
   Could this be a Feature level tag instead?



##########
extensions/elasticsearch/ElasticsearchCredentialsControllerService.cpp:
##########
@@ -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.
+ */
+
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/Resource.h"
+#include "core/PropertyBuilder.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+const core::Property ElasticsearchCredentialsControllerService::Username = core::PropertyBuilder::createProperty("Username")
+    ->withDescription("The username for basic authentication")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property ElasticsearchCredentialsControllerService::Password = core::PropertyBuilder::createProperty("Password")
+    ->withDescription("The password for basic authentication")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property ElasticsearchCredentialsControllerService::ApiKey = core::PropertyBuilder::createProperty("API Key")
+    ->withDescription("The API Key to use")
+    ->build();
+
+
+void ElasticsearchCredentialsControllerService::initialize() {
+  setSupportedProperties(properties());
+}
+
+void ElasticsearchCredentialsControllerService::onEnable() {
+  getProperty(ApiKey.getName(), api_key_);
+  std::string username, password;
+  getProperty(Username.getName(), username);
+  getProperty(Password.getName(), password);
+  if (!username.empty() && !password.empty())
+    username_password_.emplace(std::move(username), std::move(password));
+  if (!api_key_ && !username_password_)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Either an API Key or Username and Password must be provided");

Review Comment:
   Should we maybe throw an exception if both are set, or it should prioritize between API key and basic authentication? In the latter case maybe we should note it in the description.



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }

Review Comment:
   Should we handle additional types somehow or just add a log message?



##########
docker/test/integration/features/opensearch.feature:
##########
@@ -0,0 +1,100 @@
+# 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.
+
+Feature: PostElasticsearch works on Opensearch (Opensearch doesnt support API Keys)
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Opensearch container requires more RAM than what the CI environment has

Review Comment:
   Could this be a Feature level tag instead?



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();

Review Comment:
   Should we yield here? Even if the `flowfiles_in_payload` is empty there may have been several other flowfiles sent to the Failure relationship, so the queue wasn't necessarily empty.



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+  Scenario: MiNiFi instance indexes a document on Elasticsearch using Basic Authentication
+    Given an Elasticsearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Elasticsearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Elasticsearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+  Scenario: MiNiFi instance creates a document on Elasticsearch using Basic Authentication

Review Comment:
   This should say "authenticating with API Key"



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {

Review Comment:
   I may extract some parts like `collectFlowFiles` and `processSubmitResults`, but I don't insist if you think this way it's more readable.



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,102 @@
+/**
+ * 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 "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");
+  minifi::test::SingleProcessorTestController test_controller{put_elasticsearch_json};
+  auto elasticsearch_credentials_controller_service = test_controller.plan->addController("ElasticsearchCredentialsControllerService", "elasticsearch_credentials_controller_service");
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                     PostElasticsearch::ElasticCredentials.getName(),
+                                     "elasticsearch_credentials_controller_service"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Hosts.getName(),
+                                    "localhost:10433"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Action.getName(),
+                                    "${elastic_action}"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Index.getName(),
+                                    "test_index"));
+
+  SECTION("Index with valid basic authentication") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Username.getName(),
+                                            MockElasticAuthHandler::USERNAME));
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Password.getName(),
+                                            MockElasticAuthHandler::PASSWORD));
+
+    std::vector<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> inputs;
+
+    auto results = test_controller.trigger({std::make_tuple<const std::string_view, std::unordered_map<std::string, std::string>>(R"({"field1":"value1"}")", {{"elastic_action", "index"}}),
+                                            std::make_tuple<const std::string_view, std::unordered_map<std::string, std::string>>(R"({"field1":"value2"}")", {{"elastic_action", "index"}})});
+    REQUIRE(results[PostElasticsearch::Success].size() == 2);
+    for (const auto& result : results[PostElasticsearch::Success]) {
+      auto attributes = result->getAttributes();
+      CHECK(attributes.contains("elasticsearch.index._id"));
+      CHECK(attributes.contains("elasticsearch.index._index"));
+    }
+  }
+
+  SECTION("Update with valid ApiKey") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::ApiKey.getName(),
+                                            MockElasticAuthHandler::API_KEY));
+    CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                            PostElasticsearch::Identifier.getName(),
+                                            "${filename}"));
+
+    auto results = test_controller.trigger(R"({"field1":"value1"}")", {{"elastic_action", "upsert"}});
+    REQUIRE(results[PostElasticsearch::Success].size() == 1);
+    auto attributes = results[PostElasticsearch::Success][0]->getAttributes();
+    CHECK(attributes.contains("elasticsearch.update._id"));
+    CHECK(attributes.contains("elasticsearch.update._index"));
+  }
+
+  SECTION("Invalid ApiKey") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::ApiKey.getName(),
+                                            "invalid_api_key"));
+
+    auto results = test_controller.trigger(R"({"field1":"value1"}")", {{"elastic_action", "create"}});
+    CHECK(results[PostElasticsearch::Failure].size() == 1);
+  }
+
+  SECTION("Invalid basic authentication") {

Review Comment:
   Could you add test for Error relationship case as well?



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();
+    return;
+  }
+
+
+  client_.setPostFields(payload.str());
+  auto result = submitRequest(client_, flowfiles_in_payload.size());
+  if (!result) {
+    logger_->log_error(result.error().c_str());
+    for (const auto& flow_file_in_payload: flowfiles_in_payload)
+      session->transfer(flow_file_in_payload, Failure);
+    return;
+  }
+
+  auto& items = result->operator[]("items");
+  gsl_Expects(items.Size() == flowfiles_in_payload.size());
+  for (size_t i = 0; i < items.Size(); ++i) {
+    for (auto it = items[i].MemberBegin(); it != items[i].MemberEnd(); ++it) {
+      addAttributesFromResponse("elasticsearch", it, *flowfiles_in_payload[i]);
+    }
+    if (items[i].MemberBegin()->value.HasMember("error"))

Review Comment:
   If an `error` attribute is present is there any other attribute that is received and set in the flowfile? Do we want to do that in the error case? I see that in Nifi's PutElasticsearchJson processor for example a specific `elasticsearch.put.error` attribute is set if an error occurs.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r901568535


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,300 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  }
+}
+}  // namespace
+
+void PostElasticsearch::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
+  gsl_Expects(context && session && max_batch_size_ > 0);
+  std::stringstream payload;
+  std::vector<std::shared_ptr<core::FlowFile>> flowfiles_in_payload;
+  for (size_t flow_files_processed = 0; flow_files_processed < max_batch_size_; ++flow_files_processed) {
+    auto flow_file = session->get();
+    if (!flow_file)
+      break;
+    auto elastic_payload = ElasticPayload::parse(*session, *context, flow_file);
+    if (!elastic_payload) {
+      logger_->log_error(elastic_payload.error().c_str());
+      session->transfer(flow_file, Failure);
+      continue;
+    }
+
+    payload << elastic_payload->toString() << "\n";
+    flowfiles_in_payload.push_back(flow_file);
+  }
+
+  if (flowfiles_in_payload.empty()) {
+    yield();

Review Comment:
   You are right, I've removed the yield. https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR297



##########
extensions/elasticsearch/ElasticsearchCredentialsControllerService.cpp:
##########
@@ -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.
+ */
+
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/Resource.h"
+#include "core/PropertyBuilder.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+const core::Property ElasticsearchCredentialsControllerService::Username = core::PropertyBuilder::createProperty("Username")
+    ->withDescription("The username for basic authentication")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property ElasticsearchCredentialsControllerService::Password = core::PropertyBuilder::createProperty("Password")
+    ->withDescription("The password for basic authentication")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property ElasticsearchCredentialsControllerService::ApiKey = core::PropertyBuilder::createProperty("API Key")
+    ->withDescription("The API Key to use")
+    ->build();
+
+
+void ElasticsearchCredentialsControllerService::initialize() {
+  setSupportedProperties(properties());
+}
+
+void ElasticsearchCredentialsControllerService::onEnable() {
+  getProperty(ApiKey.getName(), api_key_);
+  std::string username, password;
+  getProperty(Username.getName(), username);
+  getProperty(Password.getName(), password);
+  if (!username.empty() && !password.empty())
+    username_password_.emplace(std::move(username), std::move(password));
+  if (!api_key_ && !username_password_)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Either an API Key or Username and Password must be provided");

Review Comment:
   You are right, especially since the wording of the exception implies XOR relationship between them.
   Changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-29661c58cd1e31450bbe3804f5007b4964c595b8ef1830b36c045b12cbb642c5R51



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+  Scenario: MiNiFi instance indexes a document on Elasticsearch using Basic Authentication
+    Given an Elasticsearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Elasticsearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Elasticsearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+  Scenario: MiNiFi instance creates a document on Elasticsearch using Basic Authentication

Review Comment:
   good catch, fixed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-0256a991bbecdae5406fe625e6bca90bfbfe4c3b6390f8749b63658534c80779R40



##########
docker/test/integration/resources/elasticsearch/Dockerfile:
##########
@@ -0,0 +1,7 @@
+FROM elasticsearch:8.2.2
+ADD elasticsearch.yml /usr/share/elasticsearch/config/elasticsearch.yml

Review Comment:
   good to know, I've changed it in 
   https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-a10da10e9ba31d953619871d6557f4fe8bffa0b1ceb54bf8612b6de6bc1f33e1R2-R7
   https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-cadde06b7a27fa7f57952c1aa5d678bbf39db566caa0e4b2eccd094c8821fefcR2-R5



##########
docker/test/integration/features/opensearch.feature:
##########
@@ -0,0 +1,100 @@
+# 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.
+
+Feature: PostElasticsearch works on Opensearch (Opensearch doesnt support API Keys)
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Opensearch container requires more RAM than what the CI environment has

Review Comment:
   good idea :+1: https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-9f20f0de056cfd00e2e8fa2e58c861a9f213f9c15f0f2bbcb5313feb0862c500R16



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci  # Elasticsearch container requires more RAM than what the CI environment has

Review Comment:
   good idea :+1: https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#diff-0256a991bbecdae5406fe625e6bca90bfbfe4c3b6390f8749b63658534c80779R16



-- 
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 pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
martinzink commented on PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#issuecomment-1155217976

   > Can this work with the OpenSearch fork?
   
   It should work since the API point is the same on elastic and opensearch.
   
   https://opensearch.org/docs/latest/opensearch/rest-api/document-apis/bulk/
   https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html
   
   I'll create some docker tests to verify 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] martinzink commented on a diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907253717


##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -297,6 +301,8 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   std::chrono::milliseconds keep_alive_idle_{-1};
 
+  std::optional<std::pair<std::string, std::string>> username_password_;

Review Comment:
   yeah it does help the readability changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5R304-R311



##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -57,6 +58,14 @@ class SingleProcessorTestController : public TestController {
     return trigger();
   }
 
+  auto trigger(std::initializer_list<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> flow_files) {

Review Comment:
   good idea, I've changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-a019248b6c700d2a0bc69681c6ea011e5afe07861a32a021f992375eeda11847R70



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");
+  minifi::test::SingleProcessorTestController test_controller{put_elasticsearch_json};
+  auto elasticsearch_credentials_controller_service = test_controller.plan->addController("ElasticsearchCredentialsControllerService", "elasticsearch_credentials_controller_service");
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                     PostElasticsearch::ElasticCredentials.getName(),
+                                     "elasticsearch_credentials_controller_service"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Hosts.getName(),
+                                    "localhost:10433"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Action.getName(),
+                                    "${elastic_action}"));
+  CHECK(test_controller.plan->setProperty(put_elasticsearch_json,
+                                    PostElasticsearch::Index.getName(),
+                                    "test_index"));
+
+  SECTION("Index with valid basic authentication") {
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Username.getName(),
+                                            MockElasticAuthHandler::USERNAME));
+    CHECK(test_controller.plan->setProperty(elasticsearch_credentials_controller_service,
+                                            ElasticsearchCredentialsControllerService::Password.getName(),
+                                            MockElasticAuthHandler::PASSWORD));
+
+    std::vector<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> inputs;

Review Comment:
   good catch, removed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-da99b94efb790428cd1298692800cab09ee2b398538b67ec7f0d5e4bfae2605dL53-L56



##########
extensions/elasticsearch/tests/PostElasticsearchTests.cpp:
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../PostElasticsearch.h"
+#include "../ElasticsearchCredentialsControllerService.h"
+#include "MockElastic.h"
+#include "SingleProcessorTestController.h"
+#include "Catch.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch::test {
+
+TEST_CASE("PostElasticsearch", "[elastic]") {
+  MockElastic mock_elastic("10433");
+
+  std::shared_ptr<PostElasticsearch> put_elasticsearch_json = std::make_shared<PostElasticsearch>("PostElasticsearch");

Review Comment:
   :+1: https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-da99b94efb790428cd1298692800cab09ee2b398538b67ec7f0d5e4bfae2605dR29



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static auto parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) -> nonstd::expected<ElasticPayload, std::string> {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+auto submitRequest(utils::HTTPClient& client, std::string&& payload, const size_t expected_items) -> nonstd::expected<rapidjson::Document, std::string> {

Review Comment:
   makes sense, changed this and other occurrence in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR216



##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of Syslog events to process at a time.")

Review Comment:
   changed this in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR46



##########
docker/test/integration/resources/elasticsearch/Dockerfile:
##########
@@ -0,0 +1,7 @@
+FROM elasticsearch:8.2.2
+COPY elasticsearch.yml /usr/share/elasticsearch/config/elasticsearch.yml
+COPY certs/elastic_cert.key /usr/share/elasticsearch/config/certs/elastic_cert.key
+COPY certs/elastic_cert.pem /usr/share/elasticsearch/config/certs/elastic_cert.pem
+COPY certs/elastic_transport.key /usr/share/elasticsearch/config/certs/elastic_transport.key
+COPY certs/elastic_transport.pem /usr/share/elasticsearch/config/certs/elastic_transport.pem

Review Comment:
   good idea, I've changed them in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-a10da10e9ba31d953619871d6557f4fe8bffa0b1ceb54bf8612b6de6bc1f33e1L3-L7



##########
PROCESSORS.md:
##########
@@ -1640,6 +1641,35 @@ In the list below, the names of required properties appear in bold. Any other pr
 | success | All files are routed to success |
 
 
+## PostElasticsearch
+
+### Description
+
+An Elasticsearch/Opensearch post processor that uses the Elasticsearch/Opensearch _bulk REST API.
+### Properties
+
+In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
+
+| Name                                           | Default Value | Allowable Values | Description                                                                                                                                                                                                                                                                                               |
+|------------------------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Action                                         |               |                  | The type of the operation used to index (create, delete, index, update, upsert)<br/>**Supports Expression Language: true**                                                                                                                                                                                |
+| Max Batch Size                                 | 100           |                  | The maximum number of Syslog events to process at a time.                                                                                                                                                                                                                                                 |

Review Comment:
   oops, nice catch, I've included this in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fdb1bf2a34423815cfe2cb395e83dcee3bc2f1cd#diff-fd2410931e7fdc4bf8b3ce23f5f7a27c7aacdf9337320626d86d806448c90b9bR1656



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908196278


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static nonstd::expected<ElasticPayload, std::string> parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());

Review Comment:
   note: if we are sure that the string outlives the document omitting the allocator here would only reference the string and not copy 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] martinzink commented on a diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907636408


##########
docker/test/integration/steps/steps.py:
##########
@@ -410,6 +411,62 @@ def step_impl(context):
     context.test.acquire_container("fake-gcs-server", "fake-gcs-server")
 
 
+# elasticsearch
+@given('an Elasticsearch server is set up and running')
+@given('an Elasticsearch server is set up and a single document is present with "preloaded_id" in "my_index"')
+@given('an Elasticsearch server is set up and a single document is present with "preloaded_id" in "my_index" with "value1" in "field1"')
+def step_impl(context):
+    context.test.start_elasticsearch()
+    context.test.create_doc_elasticsearch("elasticsearch", "my_index", "preloaded_id")
+
+
+# opensearch
+@given('an Opensearch server is set up and running')
+@given('an Opensearch server is set up and a single document is present with "preloaded_id" in "my_index"')
+@given('an Opensearch server is set up and a single document is present with "preloaded_id" in "my_index" with "value1" in "field1"')
+def step_impl(context):
+    context.test.start_opensearch()
+    context.test.add_elastic_user_to_opensearch("opensearch")
+    context.test.create_doc_elasticsearch("opensearch", "my_index", "preloaded_id")
+
+
+@given(u'a SSL context service is set up for PostElasticsearch and Elasticsearch')
+def step_impl(context):
+    minifi_crt_file = '/tmp/resources/elasticsearch/minifi_client.crt'
+    minifi_key_file = '/tmp/resources/elasticsearch/minifi_client.key'
+    root_ca_crt_file = '/tmp/resources/elasticsearch/root_ca.crt'
+    ssl_context_service = SSLContextService(cert=minifi_crt_file, ca_cert=root_ca_crt_file, key=minifi_key_file)
+    put_elasticsearch_json = context.test.get_node_by_name("PostElasticsearch")
+    put_elasticsearch_json.controller_services.append(ssl_context_service)
+    put_elasticsearch_json.set_property("SSL Context Service", ssl_context_service.name)

Review Comment:
   good catch, I renamed the processor later in the dev cycle... fixed in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/fa725ffb8fc830c7cf8518f40334d7ee9fa0df13#diff-dce84c359f2f7128e501f5a322d8f6ac3325c2b471844f2c42cbef9abca58185R439-R441



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907394113


##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -57,6 +58,14 @@ class SingleProcessorTestController : public TestController {
     return trigger();
   }
 
+  auto trigger(std::initializer_list<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> flow_files) {

Review Comment:
   Thanks!
   
   These kinds of issues are only optical, but can't cause real issues. The compiler (or even your editor/IDE) catches them all, early.



-- 
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 commented on a diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907494696


##########
bstrp_functions.sh:
##########
@@ -386,6 +386,7 @@ show_supported_features() {
   echo "AB. Kubernetes Support .........$(print_feature_status KUBERNETES_ENABLED)"
   echo "AC. Google Cloud Support .......$(print_feature_status GCP_ENABLED)"
   echo "AD. ProcFs Support .............$(print_feature_status PROCFS_ENABLED)"
+  echo "AE. Elasticsearch Support ......$(print_feature_status ELASTIC_ENABLED)"

Review Comment:
   lines 412 and 467 need to be updated



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,93 @@
+# 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.
+
+@no-ci  # Elasticsearch container requires more RAM than what the CI environment has
+Feature: Managing documents on Elasticsearch with PostElasticsearch
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: MiNiFi instance indexes a document on Elasticsearch using Basic Authentication
+    Given an Elasticsearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Elasticsearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Elasticsearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  Scenario: MiNiFi instance creates a document on Elasticsearch using API Key authentication

Review Comment:
   Are these two scenarios identical except for `Basic Authentication` / `ApiKey`?  If so, then they could be merged into a single Scenario Outline with two Examples.



##########
docker/DockerVerify.sh:
##########
@@ -67,7 +67,7 @@ TEST_DIRECTORY="${docker_dir}/test/integration"
 export TEST_DIRECTORY
 
 # Add --no-logcapture to see logs interleaved with the test output
-BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers)
+BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers --tags ~@no-ci)

Review Comment:
   Can we have a command-line argument or environment variable to control this?  I would like `make docker-verify` to include these tests on my computer, but skip them in CI jobs.



##########
docker/test/integration/steps/steps.py:
##########
@@ -410,6 +411,62 @@ def step_impl(context):
     context.test.acquire_container("fake-gcs-server", "fake-gcs-server")
 
 
+# elasticsearch
+@given('an Elasticsearch server is set up and running')
+@given('an Elasticsearch server is set up and a single document is present with "preloaded_id" in "my_index"')
+@given('an Elasticsearch server is set up and a single document is present with "preloaded_id" in "my_index" with "value1" in "field1"')
+def step_impl(context):
+    context.test.start_elasticsearch()
+    context.test.create_doc_elasticsearch("elasticsearch", "my_index", "preloaded_id")
+
+
+# opensearch
+@given('an Opensearch server is set up and running')
+@given('an Opensearch server is set up and a single document is present with "preloaded_id" in "my_index"')
+@given('an Opensearch server is set up and a single document is present with "preloaded_id" in "my_index" with "value1" in "field1"')
+def step_impl(context):
+    context.test.start_opensearch()
+    context.test.add_elastic_user_to_opensearch("opensearch")
+    context.test.create_doc_elasticsearch("opensearch", "my_index", "preloaded_id")
+
+
+@given(u'a SSL context service is set up for PostElasticsearch and Elasticsearch')
+def step_impl(context):
+    minifi_crt_file = '/tmp/resources/elasticsearch/minifi_client.crt'
+    minifi_key_file = '/tmp/resources/elasticsearch/minifi_client.key'
+    root_ca_crt_file = '/tmp/resources/elasticsearch/root_ca.crt'
+    ssl_context_service = SSLContextService(cert=minifi_crt_file, ca_cert=root_ca_crt_file, key=minifi_key_file)
+    put_elasticsearch_json = context.test.get_node_by_name("PostElasticsearch")
+    put_elasticsearch_json.controller_services.append(ssl_context_service)
+    put_elasticsearch_json.set_property("SSL Context Service", ssl_context_service.name)

Review Comment:
   very minor, but why are the variables containing the `PostElasticsearch` processor called `put_elastic...` instead of `post_elastic...` everywhere in this file?



##########
docker/test/integration/features/opensearch.feature:
##########
@@ -0,0 +1,97 @@
+# 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.
+
+@no-ci  # Opensearch container requires more RAM than what the CI environment has
+Feature: PostElasticsearch works on Opensearch (Opensearch doesnt support API Keys)
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: MiNiFi instance indexes a document on Opensearch using Basic Authentication
+    Given an Opensearch server is set up and running
+    And a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with the content "{ "field1" : "value1" }" is present in "/tmp/input"
+    And a PostElasticsearch processor
+    And the "Hosts" property of the PostElasticsearch processor is set to "https://opensearch:9200"
+    And the "Index" property of the PostElasticsearch processor is set to "my_index"
+    And the "Identifier" property of the PostElasticsearch processor is set to "my_id"
+    And the "Action" property of the PostElasticsearch processor is set to "index"
+    And a SSL context service is set up for PostElasticsearch and Opensearch
+    And an ElasticsearchCredentialsService is set up for PostElasticsearch with Basic Authentication
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the PostElasticsearch
+    And the "success" relationship of the PostElasticsearch processor is connected to the PutFile
+
+    When both instances start up
+    Then a flowfile with the content "{ "field1" : "value1" }" is placed in the monitored directory in less than 20 seconds
+    And Opensearch has a document with "my_id" in "my_index" that has "value1" set in "field1"
+
+  Scenario: MiNiFi instance creates a document on Opensearch using Basic Authentication

Review Comment:
   Here, too, it looks like the only difference between the first two Scenarios is the value of the Action property, so they could be merged into a single Scenario Outline.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907633624


##########
docker/DockerVerify.sh:
##########
@@ -67,7 +67,7 @@ TEST_DIRECTORY="${docker_dir}/test/integration"
 export TEST_DIRECTORY
 
 # Add --no-logcapture to see logs interleaved with the test output
-BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers)
+BEHAVE_OPTS=(-f pretty --logging-level INFO --logging-clear-handlers --tags ~@no-ci)

Review Comment:
   Good idea, fyi I'm currently working on revamping docker-verify in MINIFICPP-1874 so I would do this properly in that ticket if thats okay.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907261175


##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -57,6 +58,14 @@ class SingleProcessorTestController : public TestController {
     return trigger();
   }
 
+  auto trigger(std::initializer_list<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> flow_files) {

Review Comment:
   If you remove the constructor, then you can use aggregate initialization without having to spell out the type name of the flow file data, and without implicit constructors. The downside is that I think you need to add an empty pair of braces for the attributes every time, but maybe even that is optional if you add a default member initializer.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r896860536


##########
CMakeLists.txt:
##########
@@ -116,7 +116,9 @@ option(ENABLE_OPENWSMAN "Enables the Openwsman extensions." OFF)
 option(ENABLE_AZURE "Enables Azure support." ON)
 option(ENABLE_ENCRYPT_CONFIG "Enables build of encrypt-config binary." ON)
 option(ENABLE_SPLUNK "Enable Splunk support" ON)
+option(ENABLE_ELASTICSEARCH "Enable Elasticsearch support" OFF)
 option(ENABLE_GCP "Enable Google Cloud support" ON)
+option(ENABLE_PROCFS "Enable ProcFs support" OFF)

Review Comment:
   It used to be this way on main (didn't check the origin, probably a messed up merge), and I've recently removed the duplicate in #1341. Maybe you saw the diff only and added it back or merged from an older version?



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PutElasticsearchJson

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r895893981


##########
CMakeLists.txt:
##########
@@ -116,7 +116,9 @@ option(ENABLE_OPENWSMAN "Enables the Openwsman extensions." OFF)
 option(ENABLE_AZURE "Enables Azure support." ON)
 option(ENABLE_ENCRYPT_CONFIG "Enables build of encrypt-config binary." ON)
 option(ENABLE_SPLUNK "Enable Splunk support" ON)
+option(ENABLE_ELASTICSEARCH "Enable Elasticsearch support" OFF)
 option(ENABLE_GCP "Enable Google Cloud support" ON)
+option(ENABLE_PROCFS "Enable ProcFs support" OFF)

Review Comment:
   This is already added on line 127



##########
docker/test/integration/features/elasticsearch.feature:
##########
@@ -0,0 +1,96 @@
+# 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.
+
+Feature: Sending data to Splunk HEC using PutSplunkHTTP
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  @no-ci

Review Comment:
   Please add a comment about why this can't run in the CI.



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r901569542


##########
.github/workflows/ci.yml:
##########
@@ -166,7 +166,7 @@ jobs:
           cmake -DUSE_SHARED_LIBS=ON -DCMAKE_BUILD_TYPE=Release -DSTRICT_GSL_CHECKS=AUDIT -DFAIL_ON_WARNINGS=ON -DENABLE_AWS=ON -DENABLE_AZURE=ON -DENABLE_BUSTACHE=ON -DENABLE_COAP=ON \
               -DENABLE_ENCRYPT_CONFIG=ON -DENABLE_GPS=ON -DENABLE_JNI=ON -DENABLE_LIBRDKAFKA=ON -DENABLE_LINTER=ON -DENABLE_MQTT=ON -DENABLE_NANOFI=ON -DENABLE_OPC=ON -DENABLE_OPENCV=ON \
               -DENABLE_OPENWSMAN=ON -DENABLE_OPS=ON -DENABLE_PCAP=ON -DENABLE_PYTHON=ON -DENABLE_SENSORS=ON -DENABLE_SFTP=ON -DENABLE_SQL=ON -DENABLE_SYSTEMD=ON -DENABLE_TENSORFLOW=OFF \
-              -DENABLE_USB_CAMERA=ON -DENABLE_SCRIPTING=ON -DENABLE_LUA_SCRIPTING=ON -DENABLE_KUBERNETES=ON -DENABLE_GCP=ON -DENABLE_PROCFS=ON -DCMAKE_EXPORT_COMPILE_COMMANDS=ON ..
+              -DENABLE_USB_CAMERA=ON -DENABLE_SCRIPTING=ON -DENABLE_LUA_SCRIPTING=ON -DENABLE_KUBERNETES=ON -DENABLE_GCP=ON -DENABLE_PROCFS=ON -DENABLE_ELASTICSEARCH=ON -DCMAKE_EXPORT_COMPILE_COMMANDS=ON ..

Review Comment:
   I've added to ubuntu_20_04 and windows_VS2019 ci-s in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/106414c9218d5c359bb4c10bf19ab81b576757c0#



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908575909


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static nonstd::expected<ElasticPayload, std::string> parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());
+    operation_request.AddMember("_index", index_json, first_line.GetAllocator());
+
+    if (id_) {
+      auto id_json = rapidjson::Value(id_->data(), id_->size(), first_line.GetAllocator());
+      operation_request.AddMember("_id", id_json, first_line.GetAllocator());
+    }
+
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    first_line.Accept(writer);
+
+    return buffer.GetString();
+  }
+
+  std::string operation_;
+  std::string index_;
+  std::optional<std::string> id_;
+  std::optional<rapidjson::Document> payload_;
+};
+
+nonstd::expected<rapidjson::Document, std::string> submitRequest(utils::HTTPClient& client, std::string&& payload, const size_t expected_items) {
+  client.setPostFields(std::move(payload));
+  if (!client.submit())
+    return nonstd::make_unexpected("Submit failed");
+  auto response_code = client.getResponseCode();
+  if (response_code != 200)
+    return nonstd::make_unexpected("Error occurred: " + std::to_string(response_code) + ", " + client.getResponseBody().data());
+  rapidjson::Document response;
+  rapidjson::ParseResult parse_result = response.Parse<rapidjson::kParseStopWhenDoneFlag>(client.getResponseBody().data());
+  if (parse_result.IsError())
+    return nonstd::make_unexpected("Response is not valid json");
+  if (!response.HasMember("items"))
+    return nonstd::make_unexpected("Response is invalid");
+  if (response["items"].Size() != expected_items)
+    return nonstd::make_unexpected("The number of responses dont match the number of requests");
+
+  return response;
+}
+
+void addAttributesFromResponse(std::string name, rapidjson::Value::ConstMemberIterator object, core::FlowFile& flow_file) {
+  name = name + "." + object->name.GetString();
+
+  if (object->value.IsObject()) {
+    for (auto it = object->value.MemberBegin(); it != object->value.MemberEnd(); ++it) {
+      addAttributesFromResponse(name, it, flow_file);
+    }
+  } else if (object->value.IsInt64()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetInt64()));
+  } else if (object->value.IsString()) {
+    flow_file.addAttribute(name, object->value.GetString());
+  } else if (object->value.IsBool()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetBool()));
+  } else if (object->value.IsDouble()) {
+    flow_file.addAttribute(name, std::to_string(object->value.GetDouble()));
+  } else {
+    core::logging::LoggerFactory<PostElasticsearch>::getLogger()->log_error("Unexpected %s in response json", object->value.GetType());
+  }
+}
+
+void processResponseFromElastic(const rapidjson::Document& response, core::ProcessSession& session, const std::vector<std::shared_ptr<core::FlowFile>>& flowfiles_sent) {
+  gsl_Expects(response.HasMember("items"));
+  auto& items = response["items"];
+  gsl_Expects(items.Size() == flowfiles_sent.size());
+  for (size_t i = 0; i < items.Size(); ++i) {
+    for (auto it = items[i].MemberBegin(); it != items[i].MemberEnd(); ++it) {

Review Comment:
   good idea https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/f260dc059908a52310b14d6dbd09c9dbbe81fab5#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR264



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908577439


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");
+  credentials_service->authenticateClient(client_);
+}
+
+namespace {
+
+class ElasticPayload {
+ public:
+  [[nodiscard]] std::string toString() const {
+    auto result = headerString();
+    if (payload_) {
+      rapidjson::StringBuffer payload_buffer;
+      rapidjson::Writer<rapidjson::StringBuffer> payload_writer(payload_buffer);
+      payload_->Accept(payload_writer);
+      result = result + std::string("\n") + payload_buffer.GetString();
+    }
+    return result;
+  }
+
+  static nonstd::expected<ElasticPayload, std::string> parse(core::ProcessSession& session, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) {
+    auto action = context.getProperty(PostElasticsearch::Action, flow_file);
+    if (!action || (action != "index" && action != "create" && action != "delete" && action != "update" && action != "upsert"))
+      return nonstd::make_unexpected("Missing or invalid action");
+
+    auto index = context.getProperty(PostElasticsearch::Index, flow_file);
+    if (!index)
+      return nonstd::make_unexpected("Missing index");
+
+    auto id = context.getProperty(PostElasticsearch::Identifier, flow_file);
+    if (!id && (action == "delete" || action == "update" || action == "upsert"))
+      return nonstd::make_unexpected("Identifier is required for DELETE,UPDATE and UPSERT actions");
+
+    std::optional<rapidjson::Document> payload;
+    if (action == "index" || action == "create") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      utils::JsonInputCallback callback(*payload);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+    }
+    if (action == "update" || action == "upsert") {
+      payload = rapidjson::Document(rapidjson::kObjectType);
+      rapidjson::Document doc_member(rapidjson::kObjectType, &payload->GetAllocator());
+      utils::JsonInputCallback callback(doc_member);
+      if (session.read(flow_file, std::ref(callback)) < 0) {
+        return nonstd::make_unexpected("invalid flowfile content");
+      }
+      if (action == "upsert") {
+        action = "update";
+        doc_member.AddMember("doc_as_upsert", true, doc_member.GetAllocator());
+      }
+      payload->AddMember("doc", doc_member, payload->GetAllocator());
+    }
+    return ElasticPayload(std::move(*action), std::move(*index), std::move(id), std::move(payload));
+  }
+
+ private:
+  ElasticPayload(std::string operation,
+                 std::string index,
+                 std::optional<std::string> id,
+                 std::optional<rapidjson::Document> payload) :
+      operation_(std::move(operation)),
+      index_(std::move(index)),
+      id_(std::move(id)),
+      payload_(std::move(payload)) {
+  }
+
+  [[nodiscard]] std::string headerString() const {
+    rapidjson::Document first_line = rapidjson::Document(rapidjson::kObjectType);
+
+    auto operation_index_key = rapidjson::Value(operation_.data(), operation_.size(), first_line.GetAllocator());
+    first_line.AddMember(operation_index_key, rapidjson::Value{rapidjson::kObjectType}, first_line.GetAllocator());
+    auto& operation_request = first_line[operation_.c_str()];
+
+    auto index_json = rapidjson::Value(index_.data(), index_.size(), first_line.GetAllocator());

Review Comment:
   good idea, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/f260dc059908a52310b14d6dbd09c9dbbe81fab5#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dL191-R202



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908575622


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::MaxBatchSize = core::PropertyBuilder::createProperty("Max Batch Size")
+    ->withDescription("The maximum number of flow files to process at a time.")
+    ->withDefaultValue<uint64_t>(100)
+    ->build();
+
+const core::Property PostElasticsearch::ElasticCredentials = core::PropertyBuilder::createProperty("Elasticsearch Credentials Provider Service")
+    ->withDescription("The Controller Service used to obtain Elasticsearch credentials.")
+    ->isRequired(true)
+    ->asType<ElasticsearchCredentialsControllerService>()
+    ->build();
+
+const core::Property PostElasticsearch::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)
+    ->asType<minifi::controllers::SSLContextService>()->build();
+
+const core::Property PostElasticsearch::Hosts = core::PropertyBuilder::createProperty("Hosts")
+    ->withDescription("A comma-separated list of HTTP hosts that host Elasticsearch query nodes. Currently only supports a single host.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Index = core::PropertyBuilder::createProperty("Index")
+    ->withDescription("The name of the index to use.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+const core::Property PostElasticsearch::Identifier = core::PropertyBuilder::createProperty("Identifier")
+    ->withDescription("If the Action is \"index\" or \"create\", this property may be left empty or evaluate to an empty value, "
+                      "in which case the document's identifier will be auto-generated by Elasticsearch. "
+                      "For all other Actions, the attribute must evaluate to a non-empty value.")
+    ->supportsExpressionLanguage(true)
+    ->build();
+
+
+void PostElasticsearch::initialize() {
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
+}
+
+namespace {
+auto getSSLContextService(core::ProcessContext& context) {
+  if (auto ssl_context = context.getProperty(PostElasticsearch::SSLContext))
+    return std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(context.getControllerService(*ssl_context));
+  return std::shared_ptr<minifi::controllers::SSLContextService>{};
+}
+
+auto getCredentialsService(core::ProcessContext& context) {
+  if (auto credentials = context.getProperty(PostElasticsearch::ElasticCredentials))
+    return std::dynamic_pointer_cast<ElasticsearchCredentialsControllerService>(context.getControllerService(*credentials));
+  return std::shared_ptr<ElasticsearchCredentialsControllerService>{};
+}
+}  // namespace
+
+void PostElasticsearch::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
+  gsl_Expects(context);
+
+  context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
+  std::string host_url{};
+  if (auto hosts_str = context->getProperty(Hosts)) {
+    auto hosts = utils::StringUtils::split(*hosts_str, ",");
+    if (hosts.size() > 1)
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Multiple hosts not yet supported");
+    host_url = hosts[0];
+  } else {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing or invalid hosts");
+  }
+
+  auto credentials_service = getCredentialsService(*context);
+  if (!credentials_service)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Missing Elasticsearch credentials service");
+
+  client_.initialize("POST", host_url + "/_bulk", getSSLContextService(*context));
+  client_.setContentType("application/json");

Review Comment:
   great catch, I reverted PostElasticsearch so it wont reuse it and I will create a ticket to make HTTPClient not single-use only.
   https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/2f44b46f16a0d1be87788574216ac730811d84a1



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r908572818


##########
extensions/elasticsearch/PostElasticsearch.cpp:
##########
@@ -0,0 +1,314 @@
+/**
+ * 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 "PostElasticsearch.h"
+#include <vector>
+#include <utility>
+
+#include "ElasticsearchCredentialsControllerService.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "rapidjson/document.h"
+#include "rapidjson/stream.h"
+#include "rapidjson/writer.h"
+#include "utils/expected.h"
+#include "utils/JsonCallback.h"
+
+namespace org::apache::nifi::minifi::extensions::elasticsearch {
+
+const core::Relationship PostElasticsearch::Success("success", "All flowfiles that succeed in being transferred into Elasticsearch go here.");
+const core::Relationship PostElasticsearch::Failure("failure", "All flowfiles that fail for reasons unrelated to server availability go to this relationship.");
+const core::Relationship PostElasticsearch::Error("error", "All flowfiles that Elasticsearch responded to with an error go to this relationship.");
+
+const core::Property PostElasticsearch::Action = core::PropertyBuilder::createProperty("Action")
+    ->withDescription("The type of the operation used to index (create, delete, index, update, upsert)")
+    ->supportsExpressionLanguage(true)
+    ->build();

Review Comment:
   good idea, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/f260dc059908a52310b14d6dbd09c9dbbe81fab5#diff-bdbf058968be4e5a837bb513eb74292c24911b773939d6b3404a7c5e7b9a609dR43-R72



-- 
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 diff in pull request #1349: MINIFICPP-1843 Implement PostElasticsearch

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1349:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1349#discussion_r907293622


##########
libminifi/test/SingleProcessorTestController.h:
##########
@@ -57,6 +58,14 @@ class SingleProcessorTestController : public TestController {
     return trigger();
   }
 
+  auto trigger(std::initializer_list<std::tuple<const std::string_view, std::unordered_map<std::string, std::string>>> flow_files) {

Review Comment:
   Firstly I was a bit afraid of line endings like `}}}});` but on second thought `}})});` is maybe even worse
   Changed it in https://github.com/apache/nifi-minifi-cpp/pull/1349/commits/d3c6be5a692940d3f38fb22a7f892af8f0cfd3fe



-- 
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