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

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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


   Implementing `AttributesToJson` processor that generates a JSON representation of the input content attributes and writes them either in the `JSONAttributes` attribute or as the flowfile content.
   
   https://issues.apache.org/jira/browse/MINIFICPP-1607
   
   -----------------------------------------------------------------------------------------
   
   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] szaszm commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);

Review comment:
       Yes, std::regex is part of the language since C++11, and the reason we couldn't use it before was supporting GCC 4.8, which has only partial support for C++11. Now we have dropped support for anything older than GCC 8, so we are good with regex.




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);

Review comment:
       we could return an optional set from `getAttributesToBeWritten` and do the "else branch" when that is `nullopt`, this way we wouldn't need to recheck the condition that makes the result "valid" outside of that method




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});

Review comment:
       I actually had a failure relationship, but it seemed to be forced and unnecessary in hindsight so I removed it in 74e8946b280c840c7c00e739f92bc5271ab69b11. NiFi has only one case where it transfers flowfile to failure and it's due to the used thirdparty JSON library's possible exception when writing the JSON object, but we do not have that here.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       It's again seems to me like a bug on the NiFi side, either in the documentation or in the code. Which one would make more sense?




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+ private:
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    explicit WriteCallback(const std::string& json_data) : json_data_(json_data) {}
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+      const auto write_ret = stream->write(reinterpret_cast<const uint8_t*>(json_data_.data()), json_data_.length());
+      return io::isError(write_ret) ? -1 : gsl::narrow<int64_t>(write_ret);
+    }
+   private:
+    std::string json_data_;
+  };
+
+  bool isCoreAttributeToBeFiltered(const std::string& attribute) const;
+  bool matchesAttributeRegex(const std::string& attribute);
+  void addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value);
+  std::string buildAttributeJsonData(std::map<std::string, std::string>&& attributes);
+
+  std::shared_ptr<logging::Logger> logger_;
+  const std::unordered_set<std::string> core_attributes_;
+  std::vector<std::string> attribute_list_;
+  std::string attributes_regular_expression_str_;
+  std::regex attributes_regular_expression_;
+  bool write_to_attribute_ = true;

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+ private:
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    explicit WriteCallback(const std::string& json_data) : json_data_(json_data) {}
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+      const auto write_ret = stream->write(reinterpret_cast<const uint8_t*>(json_data_.data()), json_data_.length());
+      return io::isError(write_ret) ? -1 : gsl::narrow<int64_t>(write_ret);
+    }
+   private:
+    std::string json_data_;
+  };
+
+  bool isCoreAttributeToBeFiltered(const std::string& attribute) const;
+  bool matchesAttributeRegex(const std::string& attribute);
+  void addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value);
+  std::string buildAttributeJsonData(std::map<std::string, std::string>&& attributes);
+
+  std::shared_ptr<logging::Logger> logger_;
+  const std::unordered_set<std::string> core_attributes_;
+  std::vector<std::string> attribute_list_;
+  std::string attributes_regular_expression_str_;
+  std::regex attributes_regular_expression_;

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::splitRemovingEmpty(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = std::regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || std::regex_search(attribute, attributes_regular_expression_);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+  if (!attribute_list_.empty()) {
+    for (const auto& attribute : attribute_list_) {
+      addAttributeToJson(root, attribute, attributes[attribute]);
+    }
+  } else {
+    for (const auto& kvp : attributes) {

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review comment:
       OK, thanks.  I would probably use `{}` instead of the extra `()`:
   ```c++
   std::string file_content(std::istreambuf_iterator<char>{is}, std::istreambuf_iterator<char>{});
   ```
   but it's fine as it is, too.




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

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

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



[GitHub] [nifi-minifi-cpp] fgerlits closed pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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


   


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

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

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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -94,10 +95,11 @@ void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSe
 }
 
 bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
-  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+  const auto& special_attributes = core::SpecialFlowAttribute::getSpecialFlowAttributes();
+  return !include_core_attributes_ && ranges::find(special_attributes, attribute) != ranges::end(special_attributes);
 }
 
-std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(core::FlowFile::AttributeMap* flowfile_attributes) const {

Review comment:
       Having a pointer parameter that we're not checking for null before dereferencing is a recipe for disaster. Can we change it to const ref? We don't seem to modify the AttributeMap.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);
+    for (const auto& key : attributes_to_write) {
+      addAttributeToJson(root, key, flowfile_attributes[key]);
+    }
+  } else {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key)) {
+        addAttributeToJson(root, key, value);
+      }
+    }
+  }
+
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  root.Accept(writer);
+  return buffer.GetString();
+}
+
+void AttributesToJSON::onTrigger(core::ProcessContext* /*context*/, core::ProcessSession* session) {
+  auto flow_file = session->get();
+  if (!flow_file) {
+    return;
+  }
+
+  auto json_data = buildAttributeJsonData(flow_file->getAttributes());

Review comment:
       Changed the parameters to `AttributeMap` pointer type and used `getAttributesPtr` instead in e2893bb98eac22b2e9199c5354b33d143494edd5




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       That's true, but should we follow that functionality? It's not really intuitive and seems more like a bug, that when we set the `Include Core Attributes` to false it does not work the same way if the attribute matches the `Attributes List` or the `Attributes Regular Expression` property. IMO if the `Include Core Attributes` is set, then those attributes should not be included. Although it would also make sense to include anything that matches either regex or the specific attributes in the attribute list.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       Updated in effc866d914fb3ccff4f70a7a0ebcea6305403fc




-- 
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] arpadboda commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);

Review comment:
       If think we can already rely on std::regex, even if the c++20 PR is not yet merged. @szaszm can you confirm?
   
   Moreover std::regex can throw in case of invalid regex. Which is fair as a throw in onschedule is going to cause a retry later, but I guess the error message is not going to be too talkative. 

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});

Review comment:
       This processor has a failure relationship in NiFi.
   Are you sure there is no such case where we can't provide proper json representation?

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || attributes_regular_expression_.match(attribute);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());

Review comment:
       What happens in case there is a spec char in the value (such as "{" or ":")? 
   Do we expect RapidJson library to escape or shall we (if we can)?




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,293 @@
+/**
+ *
+ * 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 <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    utils::putFileToDir(dir_, TEST_FILE_NAME, TEST_FILE_CONTENT);
+  }
+
+  void assertJSONAttributesFromLog(const std::unordered_map<std::string, std::optional<std::string>>& expected_attributes) {
+    auto match = LogTestController::getInstance().matchesRegex("key:JSONAttributes value:(.*)").value();
+    assertAttributes(expected_attributes, match[1].str());

Review comment:
       How clear is the exception thrown when the optional is null?  I think an explicit REQUIRE might give a better diagnostic:
   ```suggestion
       auto match = LogTestController::getInstance().matchesRegex("key:JSONAttributes value:(.*)");
       REQUIRE(match);
       assertAttributes(expected_attributes, (*match)[1].str());
   ```




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -310,6 +311,10 @@ struct SpecialFlowAttribute {
   static const std::string ALTERNATE_IDENTIFIER;
   // Flow identifier
   static const std::string FLOW_ID;
+
+  static std::unordered_set<std::string> getSpecialFlowAttributes() {
+    return {PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID};

Review comment:
       Good idea, updated in e2893bb98eac22b2e9199c5354b33d143494edd5

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,106 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "utils/Enum.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;

Review comment:
       Fixed in e2893bb98eac22b2e9199c5354b33d143494edd5




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -310,6 +311,10 @@ struct SpecialFlowAttribute {
   static const std::string ALTERNATE_IDENTIFIER;
   // Flow identifier
   static const std::string FLOW_ID;
+
+  static std::unordered_set<std::string> getSpecialFlowAttributes() {
+    return {PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID};
+  }

Review comment:
       OK, makes sense.  I thought it was already an enum, just not a smart one, but you're right, it's not.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review comment:
       Thanks it looks better this way, updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":\"\"}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes with special characters to a flowfile attribute", "[AttributesToJSONTests]") {
+  update_attribute_->setDynamicProperty("special_attribute", "\\\"");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "special_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"special_attribute\":" + escapeJson("\\\"") + "}";
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Non-existent or empty selected attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute,empty_attribute");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":null,\"empty_attribute\":null}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "All non-existent or empty attributes shall be written as null in JSON", "[AttributesToJSONTests]") {

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":\"\"}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes with special characters to a flowfile attribute", "[AttributesToJSONTests]") {
+  update_attribute_->setDynamicProperty("special_attribute", "\\\"");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "special_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"special_attribute\":" + escapeJson("\\\"") + "}";
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Non-existent or empty selected attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute,empty_attribute");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":null,\"empty_attribute\":null}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "All non-existent or empty attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":null,\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "JSON attributes are written in flowfile", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::Destination.getName(), "flowfile-content");
+  test_controller_.runSession(plan_);
+  std::string expected_content = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == expected_content.size());

Review comment:
       Updated in 4e8f82bc2e1f7997a7d51dab9ff413fc0a1d0e13




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,

Review comment:
       as this is a single use function, it would be better IMO if we moved it into the `AttributesToJSONTests.cpp` if possible, thus other test wouldn't have to parse `<regex>` (or move the impl to the `TestBase.cpp` and return a `vector<string>`, but that would sacrifice some semantic power in the name of build-time)




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);

Review comment:
       Could you elaborate a bit more please? I'm not sure what do you mean by rechecking the result of `getAttributesToBeWritten`. When would it be invalid?




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_search(key, attributes_regular_expression_.value())) {

Review comment:
       I think NiFi matches the regex against the whole attribute name, not only part of if.  I would change `regex_search` to `regex_match` so we do the same.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       I asked the NiFi team about this and they said the same, the empty string values should not be modified. I'll update according to 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] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       That's true, but should we follow that functionality? It's not really intuitive and seems more like a bug, that when we set the `Include Core Attributes` to false it does not work the same way if the attribute matches the `Attributes List` or the `Attributes Regular Expression` property. IMO if the `Include Core Attributes` is set, then those attributes should not be included.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       Updated in 16ebbb4cc0bf6256200ee79c755133dec8ae09e2. Thanks for pointing out these discrepancies!




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,

Review comment:
       The `matchesRegex` depends on the `log_output` member of the `LogTestController` to retrieve and match the log output, and there is no way currently to get the `log_output` outside of the class. We could check how it influences the build times of other tests, but if it's not that big impact, I would keep it as a utility for other use cases as well, and would not expose the `log_output`.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::splitRemovingEmpty(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = std::regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || std::regex_search(attribute, attributes_regular_expression_);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+  if (!attribute_list_.empty()) {
+    for (const auto& attribute : attribute_list_) {
+      addAttributeToJson(root, attribute, attributes[attribute]);
+    }
+  } else {
+    for (const auto& kvp : attributes) {
+      addAttributeToJson(root, kvp.first, kvp.second);
+    }
+  }

Review comment:
       Fixed the funtionality and the description in 1b513c74851d9f35ae42efd2d5788d04d7087536. Added additional tests and refactored the tests to ignore the order of the json parameters.




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       the nifi implementation (despite what the documentation says) writes `null` only if the attribute does not exist and null value is requested, it does not seem to write `null` when the value of the attribute is an empty string




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);

Review comment:
       Thanks for the explanation it is clear now, updated in effc866d914fb3ccff4f70a7a0ebcea6305403fc




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);

Review comment:
       Updated in 78d69d19c56f1eaba7718b0253acdeba672f46f4




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+ private:
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    explicit WriteCallback(const std::string& json_data) : json_data_(json_data) {}
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+      const auto write_ret = stream->write(reinterpret_cast<const uint8_t*>(json_data_.data()), json_data_.length());
+      return io::isError(write_ret) ? -1 : gsl::narrow<int64_t>(write_ret);
+    }
+   private:
+    std::string json_data_;
+  };
+
+  bool isCoreAttributeToBeFiltered(const std::string& attribute) const;
+  bool matchesAttributeRegex(const std::string& attribute);
+  void addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value);
+  std::string buildAttributeJsonData(std::map<std::string, std::string>&& attributes);
+
+  std::shared_ptr<logging::Logger> logger_;
+  const std::unordered_set<std::string> core_attributes_;
+  std::vector<std::string> attribute_list_;
+  std::string attributes_regular_expression_str_;
+  std::regex attributes_regular_expression_;
+  bool write_to_attribute_ = true;

Review comment:
       it would be nicer to have a (smart) enum-valued `destination_` of either `FLOWFILE_ATTRIBUTE` or `FLOWFILE_CONTENT` instead of a `bool write_to_attribute_`

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::splitRemovingEmpty(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = std::regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || std::regex_search(attribute, attributes_regular_expression_);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+  if (!attribute_list_.empty()) {
+    for (const auto& attribute : attribute_list_) {
+      addAttributeToJson(root, attribute, attributes[attribute]);
+    }
+  } else {
+    for (const auto& kvp : attributes) {

Review comment:
       I haven't tried this, but I think we can use structured bindings now:
   ```suggestion
       for (const auto& [key, value] : attributes) {
   ```

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";

Review comment:
       `ProcessorName` is not used anywhere.  I know almost all processors have it, but we should remove it at some point, and not add it to new processors.

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::splitRemovingEmpty(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = std::regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || std::regex_search(attribute, attributes_regular_expression_);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+  if (!attribute_list_.empty()) {
+    for (const auto& attribute : attribute_list_) {
+      addAttributeToJson(root, attribute, attributes[attribute]);
+    }
+  } else {
+    for (const auto& kvp : attributes) {
+      addAttributeToJson(root, kvp.first, kvp.second);
+    }
+  }

Review comment:
       So if both a list and a regex is given, we select an attribute if it is in the list **and** matches the regex?  I find that surprising, I would have expected **or**.  Is this how NiFi works?

##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -310,6 +311,10 @@ struct SpecialFlowAttribute {
   static const std::string ALTERNATE_IDENTIFIER;
   // Flow identifier
   static const std::string FLOW_ID;
+
+  static std::unordered_set<std::string> getSpecialFlowAttributes() {
+    return {PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID};
+  }

Review comment:
       if it doesn't require too many changes, I would prefer to convert `SpecialFlowAttribute` to a `SMART_ENUM` and use its `values()` method instead of adding this

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());

Review comment:
       Why do we check just the size?  I would replace this with `file_contents[0] == TEST_FILE_CONTENT` everywhere.  The performance difference must be tiny.

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review comment:
       why is the `()` around `std::istreambuf_iterator<char>(is)` needed?

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":\"\"}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes with special characters to a flowfile attribute", "[AttributesToJSONTests]") {
+  update_attribute_->setDynamicProperty("special_attribute", "\\\"");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "special_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"special_attribute\":" + escapeJson("\\\"") + "}";
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Non-existent or empty selected attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute,empty_attribute");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":null,\"empty_attribute\":null}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "All non-existent or empty attributes shall be written as null in JSON", "[AttributesToJSONTests]") {

Review comment:
       non-existent attributes are not written in this test case
   ```suggestion
   TEST_CASE_METHOD(AttributesToJSONTestFixture, "All empty attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
   ```

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,103 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  static const std::set<std::string> DESTINATIONS;
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+  static constexpr char const* ProcessorName = "AttributesToJSON";
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+ private:
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    explicit WriteCallback(const std::string& json_data) : json_data_(json_data) {}
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+      const auto write_ret = stream->write(reinterpret_cast<const uint8_t*>(json_data_.data()), json_data_.length());
+      return io::isError(write_ret) ? -1 : gsl::narrow<int64_t>(write_ret);
+    }
+   private:
+    std::string json_data_;
+  };
+
+  bool isCoreAttributeToBeFiltered(const std::string& attribute) const;
+  bool matchesAttributeRegex(const std::string& attribute);
+  void addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value);
+  std::string buildAttributeJsonData(std::map<std::string, std::string>&& attributes);
+
+  std::shared_ptr<logging::Logger> logger_;
+  const std::unordered_set<std::string> core_attributes_;
+  std::vector<std::string> attribute_list_;
+  std::string attributes_regular_expression_str_;
+  std::regex attributes_regular_expression_;

Review comment:
       I would remove `attributes_regular_expression_str_` and change the type of `attributes_regular_expression_` to `std::optional<std::regex>`

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());
+      file_contents.push_back(file_content);
+      return true;
+    };
+
+    utils::file::FileUtils::list_dir(dir_, callback, plan_->getLogger(), false);
+
+    return file_contents;
+  }
+
+ protected:
+  TestController test_controller_;
+  std::shared_ptr<TestPlan> plan_;
+  std::string dir_;
+  std::shared_ptr<core::Processor> getfile_;
+  std::shared_ptr<core::Processor> update_attribute_;
+  std::shared_ptr<core::Processor> attribute_to_json_;
+  std::shared_ptr<core::Processor> logattribute_;
+  std::shared_ptr<core::Processor> putfile_;
+};
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move all attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes to a flowfile attribute", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":\"\"}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes with special characters to a flowfile attribute", "[AttributesToJSONTests]") {
+  update_attribute_->setDynamicProperty("special_attribute", "\\\"");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "special_attribute");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"special_attribute\":" + escapeJson("\\\"") + "}";
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "Non-existent or empty selected attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::AttributesList.getName(), "my_attribute,non_existent_attribute,empty_attribute");
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:{\"my_attribute\":\"my_value\",\"non_existent_attribute\":null,\"empty_attribute\":null}"));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "All non-existent or empty attributes shall be written as null in JSON", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::NullValue.getName(), "true");
+  test_controller_.runSession(plan_);
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == TEST_FILE_CONTENT.size());
+  std::string expected_json = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":null,\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+  REQUIRE(LogTestController::getInstance().contains("key:JSONAttributes value:" + expected_json));
+}
+
+TEST_CASE_METHOD(AttributesToJSONTestFixture, "JSON attributes are written in flowfile", "[AttributesToJSONTests]") {
+  plan_->setProperty(attribute_to_json_, org::apache::nifi::minifi::processors::AttributesToJSON::Destination.getName(), "flowfile-content");
+  test_controller_.runSession(plan_);
+  std::string expected_content = "{\"absolute.path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator() + TEST_FILE_NAME) + ",\"empty_attribute\":\"\",\"filename\":" + escapeJson(TEST_FILE_NAME) + ",\"flow.id\":\"test\",\"my_attribute\":\"my_value\",\"other_attribute\":\"other_value\",\"path\":" + escapeJson(dir_ + utils::file::FileUtils::get_separator()) + "}";  // NOLINT
+
+  auto file_contents = getOutputFileContents();
+
+  REQUIRE(file_contents.size() == 1);
+  REQUIRE(file_contents[0].size() == expected_content.size());

Review comment:
       I would remove this line

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();

Review comment:
       you can use `putFileToDir()` in `TestUtils.h` for 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] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || attributes_regular_expression_.match(attribute);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());

Review comment:
       RapidJson escapes these characters by default, there is also a test case for this:
   ```
   TEST_CASE_METHOD(AttributesToJSONTestFixture, "Move selected attributes with special characters to a flowfile attribute", "[AttributesToJSONTests]") {
   ```




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);

Review comment:
       currently when `attribute_list_.empty() && !attributes_regular_expression_` the return value of `getAttributesToBeWritten` (which then is an empty set) is considered "invalid" in the sense that we fallback on enumerating all the attributes on the flowfile, but of course an empty set could also mean that e.g. the regex did not match any of the attributes, so we currently cannot decide on the return value of the `getAttributesToBeWritten` if we should fallback or not
   
   if we returned an optional from `getAttributesToBeWritten` the return value alone could signal the need for fallback




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       I would support writing `null` only for non-existing attributes, attributes with an empty string value could(should?) have different semantics from non-existing attributes




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: docker/test/integration/features/attributes_to_json.feature
##########
@@ -0,0 +1,15 @@
+Feature: Writing attribute data using AttributesToJSON processor
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: Write selected attribute data to file
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with filename "test_file.log" and content "test_data%" is present in "/tmp/input"

Review comment:
       I think this line was just copied from a test where `%` was testing special characters, but it is not relevant here. Removed in dc9f6aa438e986a8ae6d1bacb85ca4fc677cb175

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_search(key, attributes_regular_expression_.value())) {

Review comment:
       You are right, updated the code and the tests in dc9f6aa438e986a8ae6d1bacb85ca4fc677cb175

##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,293 @@
+/**
+ *
+ * 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 <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    utils::putFileToDir(dir_, TEST_FILE_NAME, TEST_FILE_CONTENT);
+  }
+
+  void assertJSONAttributesFromLog(const std::unordered_map<std::string, std::optional<std::string>>& expected_attributes) {
+    auto match = LogTestController::getInstance().matchesRegex("key:JSONAttributes value:(.*)").value();
+    assertAttributes(expected_attributes, match[1].str());

Review comment:
       Updated in dc9f6aa438e986a8ae6d1bacb85ca4fc677cb175

##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,
+                std::chrono::seconds timeout = std::chrono::seconds(3),
+                std::chrono::milliseconds sleep_interval = std::chrono::milliseconds(200)) {
+    if (regex_str.length() == 0) {
+      return std::nullopt;
+    }
+    auto start = std::chrono::system_clock::now();
+    bool found = false;
+    bool timed_out = false;
+    std::regex matcher_regex(regex_str);
+    std::smatch match;
+    do {
+      std::string str = log_output.str();
+      found = std::regex_search(str, match, matcher_regex);
+      auto now = std::chrono::system_clock::now();
+      timed_out = std::chrono::duration_cast<std::chrono::milliseconds>(now - start) > std::chrono::duration_cast<std::chrono::milliseconds>(timeout);

Review comment:
       Updated in dc9f6aa438e986a8ae6d1bacb85ca4fc677cb175




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,106 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "utils/Enum.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+  SMART_ENUM(WriteDestination,
+    (FLOWFILE_ATTRIBUTE, "flowfile-attribute"),
+    (FLOWFILE_CONTENT, "flowfile-content")
+  )
+
+  explicit AttributesToJSON(const std::string& name, const utils::Identifier& uuid = {})
+      : core::Processor(name, uuid),
+        logger_(logging::LoggerFactory<AttributesToJSON>::getLogger()),
+        core_attributes_(core::SpecialFlowAttribute::getSpecialFlowAttributes()) {
+  }
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+ private:
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    explicit WriteCallback(const std::string& json_data) : json_data_(json_data) {}
+    int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+      const auto write_ret = stream->write(reinterpret_cast<const uint8_t*>(json_data_.data()), json_data_.length());
+      return io::isError(write_ret) ? -1 : gsl::narrow<int64_t>(write_ret);
+    }
+   private:
+    std::string json_data_;
+  };
+
+  bool isCoreAttributeToBeFiltered(const std::string& attribute) const;
+  std::unordered_set<std::string> getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const;
+  void addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value);
+  std::string buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes);
+
+  std::shared_ptr<logging::Logger> logger_;
+  const std::unordered_set<std::string> core_attributes_;
+  std::vector<std::string> attribute_list_;
+  std::optional<std::regex> attributes_regular_expression_;
+  WriteDestination write_destination_;
+  bool include_core_attributes_ = true;
+  bool null_value_ = false;
+};
+
+REGISTER_RESOURCE(AttributesToJSON, "Generates a JSON representation of the input FlowFile Attributes. "

Review comment:
       note: after the dynamic extension PR is merged these should migrate to the source files, no need to move it now, but remind me in case I forget to do so on rebase :) 




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);

Review comment:
       currently when `attribute_list_.empty() && !attributes_regular_expression_` the return value of `getAttributesToBeWritten` (which then is an empty set) is considered "invalid" in the sense that we fallback on enumerating all the attributes on the flowfile, but of course an empty set could also mean that e.g. the regex did not match any of the attributes, so we currently cannot decide on the return value of the `getAttributesToBeWritten` if we should fallback or not
   
   if we returned an optional from `getAttributesToBeWritten` the return value alone could signal the need for fallback, and we wouldn't need to check for a condition (`!attribute_list_.empty() || attributes_regular_expression_`) that feels like an implementation detail of `getAttributesToBeWritten`




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -310,6 +311,10 @@ struct SpecialFlowAttribute {
   static const std::string ALTERNATE_IDENTIFIER;
   // Flow identifier
   static const std::string FLOW_ID;
+
+  static std::unordered_set<std::string> getSpecialFlowAttributes() {
+    return {PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID};
+  }

Review comment:
       I'm not sure it's a good idea as there are currently 69 occurrences of the `SpecialFlowAttribute` in the codebase and it would take a lot of changes. My other problem with it is that the main use case of the `SpecialFlowAttribute`s is using them as strings while getting or setting attributes so the enums would always have to be converted to strings which would make their usage harder.




-- 
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] arpadboda commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});

Review comment:
       Makes sense, thanks. 




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

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

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



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/include/core/FlowFile.h
##########
@@ -310,6 +311,10 @@ struct SpecialFlowAttribute {
   static const std::string ALTERNATE_IDENTIFIER;
   // Flow identifier
   static const std::string FLOW_ID;
+
+  static std::unordered_set<std::string> getSpecialFlowAttributes() {
+    return {PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID};

Review comment:
       We could use a static array of `const char*` or `std::string_view`, referring to the static strings from above. I want to avoid the overhead of hash tables is possible, but preferably the allocations and the need to store a copy in each processor instance as well.
   The static object below is just a flat array of a set of pointers to static storage and sizes, about 144 bytes in size.
   ```suggestion
     static const auto& getSpecialFlowAttributes() {
       static const std::array<std::string_view, 9> list{
         PATH, ABSOLUTE_PATH, FILENAME, UUID, priority, MIME_TYPE, DISCARD_REASON, ALTERNATE_IDENTIFIER, FLOW_ID
       };
       return list;
   ```

##########
File path: extensions/standard-processors/processors/AttributesToJSON.h
##########
@@ -0,0 +1,106 @@
+/**
+ * @file AttributesToJSON.h
+ * AttributesToJSON class declaration
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <vector>
+#include <string>
+#include <set>
+#include <unordered_set>
+#include <memory>
+#include <map>
+#include <regex>
+
+#include "rapidjson/document.h"
+#include "core/Processor.h"
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "utils/Enum.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class AttributesToJSON : public core::Processor {
+ public:
+  // Supported Properties
+  static const core::Property AttributesList;
+  static const core::Property AttributesRegularExpression;
+  static const core::Property Destination;
+  static const core::Property IncludeCoreAttributes;
+  static const core::Property NullValue;
+
+  // Supported Relationships
+  static core::Relationship Success;

Review comment:
       This could (and IMHO should) be `const`.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review comment:
       It's needed because of a vexing-parse warning where the `std::istreambuf_iterator<char>(is)` could be interpreted as a parameter declaration. Here is an explanation of this: https://stackoverflow.com/questions/7007817/a-confusing-detail-about-the-most-vexing-parse




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());

Review comment:
       I asked the NiFi team about this and they said the same, the empty string values should not be modified. I'll update the code according to this as well as the documentation.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       True, I checked with the NiFi team and they said that the `Include Core Attributes` should only control whether the attributes are included by default, the regex or attribute list should override 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] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,

Review comment:
       Thanks for the effort :)




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::splitRemovingEmpty(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = std::regex(attributes_regular_expression_str_);
+  }
+  write_to_attribute_ = utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), DESTINATIONS) == "flowfile-attribute";
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+bool AttributesToJSON::matchesAttributeRegex(const std::string& attribute) {
+  return attributes_regular_expression_str_.empty() || std::regex_search(attribute, attributes_regular_expression_);
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  if (isCoreAttributeToBeFiltered(key)) {
+    logger_->log_debug("Core attribute '%s' will not be included in the attributes JSON.", key);
+    return;
+  }
+  if (!matchesAttributeRegex(key)) {
+    logger_->log_debug("Attribute '%s' does not match the set regex, therefore it will not be included in the attributes JSON.", key);
+    return;
+  }
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+  if (!attribute_list_.empty()) {
+    for (const auto& attribute : attribute_list_) {
+      addAttributeToJson(root, attribute, attributes[attribute]);
+    }
+  } else {
+    for (const auto& kvp : attributes) {
+      addAttributeToJson(root, kvp.first, kvp.second);
+    }
+  }

Review comment:
       That makes sense, I think I misunderstood the meaning of the description of the regex property: "This property can be used in combination with the attributes list property.". I checked the NiFi code and the result is the union of both the regex matching attributes and the selected attribute list. Thanks for pointing it out, I will have a fix for this.




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       I feel like excluding the core attributes only makes sense when we specify neither the list nor the regex, if the user explicitly requests the attribute by name or regex, I think we should not exclude core ones




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: docker/test/integration/features/attributes_to_json.feature
##########
@@ -0,0 +1,15 @@
+Feature: Writing attribute data using AttributesToJSON processor
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: Write selected attribute data to file
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with filename "test_file.log" and content "test_data%" is present in "/tmp/input"

Review comment:
       just picking nits, but what is the purpose of the `%` sign?

##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,
+                std::chrono::seconds timeout = std::chrono::seconds(3),
+                std::chrono::milliseconds sleep_interval = std::chrono::milliseconds(200)) {
+    if (regex_str.length() == 0) {
+      return std::nullopt;
+    }
+    auto start = std::chrono::system_clock::now();
+    bool found = false;
+    bool timed_out = false;
+    std::regex matcher_regex(regex_str);
+    std::smatch match;
+    do {
+      std::string str = log_output.str();
+      found = std::regex_search(str, match, matcher_regex);
+      auto now = std::chrono::system_clock::now();
+      timed_out = std::chrono::duration_cast<std::chrono::milliseconds>(now - start) > std::chrono::duration_cast<std::chrono::milliseconds>(timeout);

Review comment:
       I don't think these `duration_cast`s are needed; `std::chrono::duration`s constructed with different units can be compared.
   ```suggestion
         timed_out = (now - start > timeout);
   ```




-- 
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 change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/tests/unit/AttributesToJSONTests.cpp
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <string>
+#include <vector>
+
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "AttributesToJSON.h"
+#include "GetFile.h"
+#include "PutFile.h"
+#include "UpdateAttribute.h"
+#include "LogAttribute.h"
+
+namespace {
+
+class AttributesToJSONTestFixture {
+ public:
+  const std::string TEST_FILE_CONTENT = "test_content";
+  const std::string TEST_FILE_NAME = "tstFile.ext";
+
+  AttributesToJSONTestFixture() {
+    LogTestController::getInstance().setTrace<TestPlan>();
+    LogTestController::getInstance().setDebug<minifi::processors::AttributesToJSON>();
+    LogTestController::getInstance().setDebug<minifi::processors::GetFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::PutFile>();
+    LogTestController::getInstance().setDebug<minifi::processors::UpdateAttribute>();
+    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
+
+    dir_ = test_controller_.createTempDirectory();
+
+    plan_ = test_controller_.createPlan();
+    getfile_ = plan_->addProcessor("GetFile", "GetFile");
+    update_attribute_ = plan_->addProcessor("UpdateAttribute", "UpdateAttribute", core::Relationship("success", "description"), true);
+    attribute_to_json_ = plan_->addProcessor("AttributesToJSON", "AttributesToJSON", core::Relationship("success", "description"), true);
+    logattribute_ = plan_->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
+    putfile_ = plan_->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
+
+    plan_->setProperty(getfile_, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir_);
+    plan_->setProperty(putfile_, org::apache::nifi::minifi::processors::PutFile::Directory.getName(), dir_);
+
+    update_attribute_->setDynamicProperty("my_attribute", "my_value");
+    update_attribute_->setDynamicProperty("other_attribute", "other_value");
+    update_attribute_->setDynamicProperty("empty_attribute", "");
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir_ << utils::file::FileUtils::get_separator() << TEST_FILE_NAME;
+    file.open(ss.str(), std::ios::out);
+    file << TEST_FILE_CONTENT;
+    file.close();
+  }
+
+  std::string escapeJson(const std::string& json) const {
+    rapidjson::StringBuffer buffer;
+    rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+    writer.String(json.c_str(), json.size());
+    return buffer.GetString();
+  }
+
+  std::vector<std::string> getOutputFileContents() {
+    std::vector<std::string> file_contents;
+
+    auto callback = [&file_contents](const std::string& path, const std::string& filename) -> bool {
+      std::ifstream is(path + utils::file::FileUtils::get_separator() + filename, std::ifstream::binary);
+      std::string file_content((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review comment:
       OK, thanks.  I would probably use `{}` instead of the extra `()`:
   ```c++
          std::string file_content(std::istreambuf_iterator<char>{is}, std::istreambuf_iterator<char>{});
   ```
   but it's fine as it is, too.




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,162 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::set<std::string> AttributesToJSON::DESTINATIONS({"flowfile-attribute", "flowfile-content"});
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "This property can be used in combination with the attributes list property.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("flowfile-attribute")
+    ->withAllowableValues<std::string>(DESTINATIONS)
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string attributes;
+  context->getProperty(AttributesList.getName(), attributes);
+  attribute_list_ = utils::StringUtils::split(attributes, ",");
+  context->getProperty(AttributesRegularExpression.getName(), attributes_regular_expression_str_);
+  if (!attributes_regular_expression_str_.empty()) {
+    attributes_regular_expression_ = utils::Regex(attributes_regular_expression_str_);

Review comment:
       Yes, when I opened this PR the GCC 4.8 support was still there, but I'll update this now.




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {

Review comment:
       as I see, in the nifi implementation the regex based match does not check if it is a core attribute or not




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: libminifi/test/TestBase.h
##########
@@ -164,6 +165,31 @@ class LogTestController {
     return found;
   }
 
+  std::optional<std::smatch> matchesRegex(const std::string &regex_str,

Review comment:
       measured an unrelated test to see if this method makes any difference, fortunately the difference is negligible (~50ms) so I think we can keep it as is




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

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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -94,10 +95,11 @@ void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSe
 }
 
 bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
-  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+  const auto& special_attributes = core::SpecialFlowAttribute::getSpecialFlowAttributes();
+  return !include_core_attributes_ && ranges::find(special_attributes, attribute) != ranges::end(special_attributes);
 }
 
-std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(core::FlowFile::AttributeMap* flowfile_attributes) const {

Review comment:
       Good point, we can safely change it as the pointer is just the returned address of the internal member. Updated in effc866d914fb3ccff4f70a7a0ebcea6305403fc




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

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

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



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1137: MINIFICPP-1607 Add AttributesToJson processor

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



##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);
+    for (const auto& key : attributes_to_write) {
+      addAttributeToJson(root, key, flowfile_attributes[key]);
+    }
+  } else {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key)) {
+        addAttributeToJson(root, key, value);
+      }
+    }
+  }
+
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  root.Accept(writer);
+  return buffer.GetString();
+}
+
+void AttributesToJSON::onTrigger(core::ProcessContext* /*context*/, core::ProcessSession* session) {
+  auto flow_file = session->get();
+  if (!flow_file) {
+    return;
+  }
+
+  auto json_data = buildAttributeJsonData(flow_file->getAttributes());

Review comment:
       as the ``attributes_` in the `FlowFile` is of type `FlatMap`, `getAttributes` constructs a new `std::map` for backwards compatibility, we should avoid this if we can IMO

##########
File path: extensions/standard-processors/processors/AttributesToJSON.cpp
##########
@@ -0,0 +1,174 @@
+/**
+ * @file AttributesToJSON.cpp
+ * AttributesToJSON class implementation
+ *
+ * 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 "AttributesToJSON.h"
+
+#include <unordered_set>
+
+#include "rapidjson/writer.h"
+#include "utils/StringUtils.h"
+#include "utils/ProcessorConfigUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const core::Property AttributesToJSON::AttributesList(
+  core::PropertyBuilder::createProperty("Attributes List")
+    ->withDescription("Comma separated list of attributes to be included in the resulting JSON. "
+                      "If this value is left empty then all existing Attributes will be included. This list of attributes is case sensitive. "
+                      "If an attribute specified in the list is not found it will be be emitted to the resulting JSON with an empty string or NULL value.")
+    ->build());
+
+const core::Property AttributesToJSON::AttributesRegularExpression(
+  core::PropertyBuilder::createProperty("Attributes Regular Expression")
+    ->withDescription("Regular expression that will be evaluated against the flow file attributes to select the matching attributes. "
+                      "Both the matching attributes and the selected attributes from the Attributes List property will be written in the resulting JSON.")
+    ->build());
+
+const core::Property AttributesToJSON::Destination(
+  core::PropertyBuilder::createProperty("Destination")
+    ->withDescription("Control if JSON value is written as a new flowfile attribute 'JSONAttributes' or written in the flowfile content. "
+                      "Writing to flowfile content will overwrite any existing flowfile content.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(WriteDestination::FLOWFILE_ATTRIBUTE))
+    ->withAllowableValues<std::string>(WriteDestination::values())
+    ->build());
+
+const core::Property AttributesToJSON::IncludeCoreAttributes(
+  core::PropertyBuilder::createProperty("Include Core Attributes")
+    ->withDescription("Determines if the FlowFile core attributes which are contained in every FlowFile should be included in the final JSON value generated.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property AttributesToJSON::NullValue(
+  core::PropertyBuilder::createProperty("Null Value")
+    ->withDescription("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty string will be placed in the JSON.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+core::Relationship AttributesToJSON::Success("success", "All FlowFiles received are routed to success");
+
+void AttributesToJSON::initialize() {
+  setSupportedProperties({
+    AttributesList,
+    AttributesRegularExpression,
+    Destination,
+    IncludeCoreAttributes,
+    NullValue
+  });
+  setSupportedRelationships({Success});
+}
+
+void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  std::string value;
+  if (context->getProperty(AttributesList.getName(), value) && !value.empty()) {
+    attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
+  }
+  if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
+    attributes_regular_expression_ = std::regex(value);
+  }
+  write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
+  context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
+  context->getProperty(NullValue.getName(), null_value_);
+}
+
+bool AttributesToJSON::isCoreAttributeToBeFiltered(const std::string& attribute) const {
+  return !include_core_attributes_ && core_attributes_.find(attribute) != core_attributes_.end();
+}
+
+std::unordered_set<std::string> AttributesToJSON::getAttributesToBeWritten(const std::map<std::string, std::string>& flowfile_attributes) const {
+  std::unordered_set<std::string> attributes;
+
+  for (const auto& attribute : attribute_list_) {
+    if (!isCoreAttributeToBeFiltered(attribute)) {
+      attributes.insert(attribute);
+    }
+  }
+
+  if (attributes_regular_expression_) {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key) && std::regex_match(key, attributes_regular_expression_.value())) {
+        attributes.insert(key);
+      }
+    }
+  }
+
+  return attributes;
+}
+
+void AttributesToJSON::addAttributeToJson(rapidjson::Document& document, const std::string& key, const std::string& value) {
+  rapidjson::Value json_key(key.c_str(), document.GetAllocator());
+  rapidjson::Value json_val;
+  if (!value.empty() || !null_value_) {
+    json_val.SetString(value.c_str(), document.GetAllocator());
+  }
+  document.AddMember(json_key, json_val, document.GetAllocator());
+}
+
+std::string AttributesToJSON::buildAttributeJsonData(std::map<std::string, std::string>&& flowfile_attributes) {
+  auto root = rapidjson::Document(rapidjson::kObjectType);
+
+  if (!attribute_list_.empty() || attributes_regular_expression_) {
+    auto attributes_to_write = getAttributesToBeWritten(flowfile_attributes);
+    for (const auto& key : attributes_to_write) {
+      addAttributeToJson(root, key, flowfile_attributes[key]);
+    }
+  } else {
+    for (const auto& [key, value] : flowfile_attributes) {
+      if (!isCoreAttributeToBeFiltered(key)) {
+        addAttributeToJson(root, key, value);
+      }
+    }
+  }
+
+  rapidjson::StringBuffer buffer;
+  rapidjson::Writer<rapidjson::StringBuffer> writer(buffer);
+  root.Accept(writer);
+  return buffer.GetString();
+}
+
+void AttributesToJSON::onTrigger(core::ProcessContext* /*context*/, core::ProcessSession* session) {
+  auto flow_file = session->get();
+  if (!flow_file) {
+    return;
+  }
+
+  auto json_data = buildAttributeJsonData(flow_file->getAttributes());

Review comment:
       as the `attributes_` in the `FlowFile` is of type `FlatMap`, `getAttributes` constructs a new `std::map` for backwards compatibility, we should avoid this if we can IMO




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

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

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