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/01/15 09:02:06 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request #976: MINIFICPP-1448 - CWEL JSON output

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


   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.

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/CWELCustomProviderTests.cpp
##########
@@ -0,0 +1,132 @@
+/**
+ * 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 "ConsumeWindowsEventLog.h"
+
+#include "core/ConfigurableComponent.h"
+#include "processors/LogAttribute.h"
+#include "processors/PutFile.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "utils/file/FileUtils.h"
+#include "rapidjson/document.h"
+
+#include "CWELTestUtils.h"
+
+// generated from the manifest file "custom-provider/unit-test-provider.man"
+// using the command "mc -um unit-test-provider.man"
+#include "custom-provider/unit-test-provider.h"
+
+namespace {
+
+struct CustomEventData {
+  std::wstring first;
+  std::wstring second;
+  std::wstring third;
+  int binary_length;
+  const unsigned char* binary_data;
+};
+
+const std::string CUSTOM_PROVIDER_NAME = "minifi_unit_test_provider";
+const std::string CUSTOM_CHANNEL = CUSTOM_PROVIDER_NAME + "/Log";
+
+bool dispatchCustomEvent(const CustomEventData& event) {
+  static auto provider_initialized = [] {
+    return EventRegisterminifi_unit_test_provider();
+  }();

Review comment:
       replaced it




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,33 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;
-  bool apply_identifier_function_;
+  bool apply_identifier_function_{false};
   std::string provenanceUri_;
   std::string computerName_;
   uint64_t maxBufferSize_{};
   DWORD lastActivityTimestamp_{};
   std::mutex cache_mutex_;
   std::map<std::string, wel::WindowsEventLogHandler > providers_;
-  uint64_t batch_commit_size_;
+  uint64_t batch_commit_size_{};
+
+  struct OutputFormat {
+    void reset() {
+      *this = OutputFormat{};
+    }
+
+    bool xml{false};
+    bool plaintext{false};
+    struct {
+      explicit operator bool() const noexcept {
+        return raw || simple || flattened;
+      }
+
+      bool raw{false};
+      bool simple{false};
+      bool flattened{false};
+    } json;
+  } output_;

Review comment:
       but the `reset()` is indeed not needed




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,170 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+namespace {
+
+rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");
+      auto& correlation = flatten ? doc : system.AddMember("Correlation", rapidjson::kObjectType, doc.GetAllocator())["Correlation"];
+      correlation.AddMember("ActivityID", rapidjson::StringRef(correlation_xml.attribute("ActivityID").value()), doc.GetAllocator());
+    }
+
+    {
+      auto execution_xml = system_xml.child("Execution");
+      auto& execution = flatten ? doc : system.AddMember("Execution", rapidjson::kObjectType, doc.GetAllocator())["Execution"];
+      execution.AddMember("ProcessID", rapidjson::StringRef(execution_xml.attribute("ProcessID").value()), doc.GetAllocator());
+      execution.AddMember("ThreadID", rapidjson::StringRef(execution_xml.attribute("ThreadID").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("Channel", rapidjson::StringRef(system_xml.child("Channel").text().get()), doc.GetAllocator());
+    system.AddMember("Computer", rapidjson::StringRef(system_xml.child("Computer").text().get()), doc.GetAllocator());
+  }
+
+  {
+    auto eventData_xml = event_xml.child("EventData");
+    // create EventData subarray even if flatten requested
+    doc.AddMember("EventData", rapidjson::kArrayType, doc.GetAllocator());
+    for (const auto& data : eventData_xml.children()) {
+      auto name_attr = data.attribute("Name");
+      rapidjson::Value item(rapidjson::kObjectType);
+      item.AddMember("Name", rapidjson::StringRef(name_attr.value()), doc.GetAllocator());
+      item.AddMember("Content", rapidjson::StringRef(data.text().get()), doc.GetAllocator());
+      item.AddMember("Type", rapidjson::StringRef(data.name()), doc.GetAllocator());
+      // we need to query EventData because a reference to it wouldn't be stable, as we
+      // possibly add members to its parent which could result in reallocation
+      doc["EventData"].PushBack(item, doc.GetAllocator());
+      // check collision
+      if (flatten && !name_attr.empty() && !doc.HasMember(name_attr.value())) {
+        doc.AddMember(rapidjson::StringRef(name_attr.value()), rapidjson::StringRef(data.text().get()), doc.GetAllocator());

Review comment:
       added big loud warning 🙂




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+static rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toRawJSON(const pugi::xml_node& root) {
+  rapidjson::Document doc;
+  if (root.type() == pugi::xml_node_type::node_document) {
+    static_cast<rapidjson::Value&>(doc) = xmlDocumentToJSON(root, doc);
+  }
+  return doc;
+}
+
+static rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");

Review comment:
       :+1: 




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,33 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;
-  bool apply_identifier_function_;
+  bool apply_identifier_function_{false};
   std::string provenanceUri_;
   std::string computerName_;
   uint64_t maxBufferSize_{};
   DWORD lastActivityTimestamp_{};
   std::mutex cache_mutex_;
   std::map<std::string, wel::WindowsEventLogHandler > providers_;
-  uint64_t batch_commit_size_;
+  uint64_t batch_commit_size_{};
+
+  struct OutputFormat {
+    void reset() {
+      *this = OutputFormat{};
+    }
+
+    bool xml{false};
+    bool plaintext{false};
+    struct {
+      explicit operator bool() const noexcept {
+        return raw || simple || flattened;
+      }
+
+      bool raw{false};
+      bool simple{false};
+      bool flattened{false};
+    } json;
+  } output_;

Review comment:
       I think the idea of list-type properties is orthogonal to this; we could have "XML,JSON" or "JSON,Plaintext" as the output format property.  A user would understand what "JSON" is; they would not know (without asking us or the support team) what JSONRaw, JSONSimple, JSONFlattened mean and how to choose between them.
   
   I understand you have had this discussion already and have made a decision, but I think it would be worth discussing it some more, as this decision causes our code to be more complex and the user interface more difficult to use, both meaning more work in the future.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,170 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+namespace {
+
+rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");
+      auto& correlation = flatten ? doc : system.AddMember("Correlation", rapidjson::kObjectType, doc.GetAllocator())["Correlation"];
+      correlation.AddMember("ActivityID", rapidjson::StringRef(correlation_xml.attribute("ActivityID").value()), doc.GetAllocator());
+    }
+
+    {
+      auto execution_xml = system_xml.child("Execution");
+      auto& execution = flatten ? doc : system.AddMember("Execution", rapidjson::kObjectType, doc.GetAllocator())["Execution"];
+      execution.AddMember("ProcessID", rapidjson::StringRef(execution_xml.attribute("ProcessID").value()), doc.GetAllocator());
+      execution.AddMember("ThreadID", rapidjson::StringRef(execution_xml.attribute("ThreadID").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("Channel", rapidjson::StringRef(system_xml.child("Channel").text().get()), doc.GetAllocator());
+    system.AddMember("Computer", rapidjson::StringRef(system_xml.child("Computer").text().get()), doc.GetAllocator());
+  }
+
+  {
+    auto eventData_xml = event_xml.child("EventData");
+    // create EventData subarray even if flatten requested
+    doc.AddMember("EventData", rapidjson::kArrayType, doc.GetAllocator());
+    for (const auto& data : eventData_xml.children()) {
+      auto name_attr = data.attribute("Name");
+      rapidjson::Value item(rapidjson::kObjectType);
+      item.AddMember("Name", rapidjson::StringRef(name_attr.value()), doc.GetAllocator());
+      item.AddMember("Content", rapidjson::StringRef(data.text().get()), doc.GetAllocator());
+      item.AddMember("Type", rapidjson::StringRef(data.name()), doc.GetAllocator());
+      // we need to query EventData because a reference to it wouldn't be stable, as we
+      // possibly add members to its parent which could result in reallocation
+      doc["EventData"].PushBack(item, doc.GetAllocator());
+      // check collision
+      if (flatten && !name_attr.empty() && !doc.HasMember(name_attr.value())) {
+        doc.AddMember(rapidjson::StringRef(name_attr.value()), rapidjson::StringRef(data.text().get()), doc.GetAllocator());

Review comment:
       added big loud warning 🙂




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());
 
-    test_controller.runSession(test_plan);
-  }
+  matchJSON(json, expected);
+}
 
-  test_plan->reset();
-  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+class JSONOutputController : public OutputFormatTestController {
+ public:
+  JSONOutputController(std::string format) : OutputFormatTestController(APPLICATION_CHANNEL, "*", std::move(format)) {}
 
-  {
+ protected:
+  void dispatchBookmarkEvent() override {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+  }
+  void dispatchCollectedEvent() override {
     reportEvent(APPLICATION_CHANNEL, "Event one");
+  }
+};
 
-    test_controller.runSession(test_plan);
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Simple correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Simple"}.run();
+  verifyJSON(event, R"json(
+    {
+      "System": {
+        "Provider": {
+          "Name": "Application"
+        },
+        "Channel": "Application"
+      },
+      "EventData": [{
+          "Type": "Data",
+          "Content": "Event one",
+          "Name": ""
+      }]
+    }
+  )json");
+}
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Raw correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Raw"}.run();
+  verifyJSON(event, R"json(
+    [
+      {
+        "name": "Event",
+        "children": [
+          {"name": "System"},
+          {
+            "name": "EventData",
+            "children": [{
+              "name": "Data",
+              "text": "Event one"
+            }] 
+          }      
+        ]
+      }
+    ]
+  )json");
+}
+
+class CustomProviderController : public OutputFormatTestController {
+ public:
+  CustomProviderController(std::string format) : OutputFormatTestController(custom_channel, "*", std::move(format)) {}
 
-    REQUIRE(LogTestController::getInstance().contains(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<EventID Qualifiers="0">14985</EventID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Level>4</Level>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Task>0</Task>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")"));
-    // the timestamp (when the event was published) goes here
-    REQUIRE(LogTestController::getInstance().contains(R"("/><EventRecordID>)"));
-    // the ID of the event goes here (a number)
-    REQUIRE(LogTestController::getInstance().contains(R"(</EventRecordID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Channel>Application</Channel><Computer>)"));
-    // the computer name goes here
-    REQUIRE(LogTestController::getInstance().contains(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)"));
+ protected:
+  void dispatchBookmarkEvent() override {
+    REQUIRE(dispatchCustomEvent({"Bookmark", "Second", "Third", 3, 12}));
+    std::this_thread::sleep_for(std::chrono::seconds{1});

Review comment:
       unfortunately even with the API we still need the sleeps




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,170 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+namespace {
+
+rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");
+      auto& correlation = flatten ? doc : system.AddMember("Correlation", rapidjson::kObjectType, doc.GetAllocator())["Correlation"];
+      correlation.AddMember("ActivityID", rapidjson::StringRef(correlation_xml.attribute("ActivityID").value()), doc.GetAllocator());
+    }
+
+    {
+      auto execution_xml = system_xml.child("Execution");
+      auto& execution = flatten ? doc : system.AddMember("Execution", rapidjson::kObjectType, doc.GetAllocator())["Execution"];
+      execution.AddMember("ProcessID", rapidjson::StringRef(execution_xml.attribute("ProcessID").value()), doc.GetAllocator());
+      execution.AddMember("ThreadID", rapidjson::StringRef(execution_xml.attribute("ThreadID").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("Channel", rapidjson::StringRef(system_xml.child("Channel").text().get()), doc.GetAllocator());
+    system.AddMember("Computer", rapidjson::StringRef(system_xml.child("Computer").text().get()), doc.GetAllocator());
+  }
+
+  {
+    auto eventData_xml = event_xml.child("EventData");
+    // create EventData subarray even if flatten requested
+    doc.AddMember("EventData", rapidjson::kArrayType, doc.GetAllocator());
+    for (const auto& data : eventData_xml.children()) {
+      auto name_attr = data.attribute("Name");
+      rapidjson::Value item(rapidjson::kObjectType);
+      item.AddMember("Name", rapidjson::StringRef(name_attr.value()), doc.GetAllocator());
+      item.AddMember("Content", rapidjson::StringRef(data.text().get()), doc.GetAllocator());
+      item.AddMember("Type", rapidjson::StringRef(data.name()), doc.GetAllocator());
+      // we need to query EventData because a reference to it wouldn't be stable, as we
+      // possibly add members to its parent which could result in reallocation
+      doc["EventData"].PushBack(item, doc.GetAllocator());
+      // check collision
+      if (flatten && !name_attr.empty() && !doc.HasMember(name_attr.value())) {
+        doc.AddMember(rapidjson::StringRef(name_attr.value()), rapidjson::StringRef(data.text().get()), doc.GetAllocator());

Review comment:
       If I understand correctly, using `rapidjson::StringRef` all over the place means that the returned json object is valid only as long as the original pugi xml object is valid and unchanged. If so, there should be a big loud warning on the function to warn the next caller about this.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument
+ *
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument

Review comment:
       I would also explain why:
   ```suggestion
    * !!WARNING!! The json document must not outlive the xml argument. For better performance,
    * the created json document stores references to values in the xml node. Accessing the
    * json document after the xml node has been destroyed results in undefined behavior.
   ```




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -252,11 +251,27 @@ void ConsumeWindowsEventLog::onSchedule(const std::shared_ptr<core::ProcessConte
   std::string mode;
   context->getProperty(OutputFormat.getName(), mode);
 
-  writeXML_ = (mode == Both || mode == XML);
-
-  writePlainText_ = (mode == Both || mode == Plaintext);
+  output_.reset();
+  if (mode == XML) {
+    output_.xml = true;
+  } else if (mode == Plaintext) {
+    output_.plaintext = true;
+  } else if (mode == Both) {
+    output_.xml = true;
+    output_.plaintext = true;
+  } else if (mode == JSONRaw) {
+    output_.json.raw = true;
+  } else if (mode == JSONSimple) {
+    output_.json.simple = true;
+  } else if (mode == JSONFlattened) {
+    output_.json.flattened = true;
+  } else {
+    // in the future this might be considered an error, but for now due to backwards
+    // compatibility we just fall through and execute the processor outputing nothing
+    // throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unrecognized output format: " + mode);

Review comment:
       I don't know of such a marker, but note that based on the allowable values this branch should be impossible to reach if the check happens during the yaml parsing, this check is currently optional and I am not sure if we plan on switching it on by default from 1.0




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {

Review comment:
       done: using anonymous namespace

##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}

Review comment:
       added xml input

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -41,6 +48,29 @@ void reportEvent(const std::string& channel, const char* message, WORD log_level
   ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
 }
 
+struct CustomEvent {
+  std::string first;
+  std::string second;
+  std::string third;
+  int binary_length;
+  int binary_data;
+};
+
+const std::string custom_provider_name = "minifi_unit_test_provider";
+const std::string custom_channel = custom_provider_name + "/Log";

Review comment:
       done

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());
 
-    test_controller.runSession(test_plan);
-  }
+  matchJSON(json, expected);
+}
 
-  test_plan->reset();
-  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+class JSONOutputController : public OutputFormatTestController {
+ public:
+  JSONOutputController(std::string format) : OutputFormatTestController(APPLICATION_CHANNEL, "*", std::move(format)) {}
 
-  {
+ protected:
+  void dispatchBookmarkEvent() override {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+  }
+  void dispatchCollectedEvent() override {
     reportEvent(APPLICATION_CHANNEL, "Event one");
+  }
+};
 
-    test_controller.runSession(test_plan);
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Simple correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Simple"}.run();
+  verifyJSON(event, R"json(
+    {
+      "System": {
+        "Provider": {
+          "Name": "Application"
+        },
+        "Channel": "Application"
+      },
+      "EventData": [{
+          "Type": "Data",
+          "Content": "Event one",
+          "Name": ""
+      }]
+    }
+  )json");
+}
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Raw correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Raw"}.run();
+  verifyJSON(event, R"json(
+    [
+      {
+        "name": "Event",
+        "children": [
+          {"name": "System"},
+          {
+            "name": "EventData",
+            "children": [{
+              "name": "Data",
+              "text": "Event one"
+            }] 
+          }      
+        ]
+      }
+    ]
+  )json");
+}
+

Review comment:
       added

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());

Review comment:
       done




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,33 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;
-  bool apply_identifier_function_;
+  bool apply_identifier_function_{false};
   std::string provenanceUri_;
   std::string computerName_;
   uint64_t maxBufferSize_{};
   DWORD lastActivityTimestamp_{};
   std::mutex cache_mutex_;
   std::map<std::string, wel::WindowsEventLogHandler > providers_;
-  uint64_t batch_commit_size_;
+  uint64_t batch_commit_size_{};
+
+  struct OutputFormat {
+    void reset() {
+      *this = OutputFormat{};
+    }
+
+    bool xml{false};
+    bool plaintext{false};
+    struct {
+      explicit operator bool() const noexcept {
+        return raw || simple || flattened;
+      }
+
+      bool raw{false};
+      bool simple{false};
+      bool flattened{false};
+    } json;
+  } output_;

Review comment:
       I think the idea of list-type properties is orthogonal to this; we could have "XML,JSON" or "JSON,Plaintext" as the output format property.  A user would understand what "JSON" is; they would not know (without asking us or the support team) what JSONRaw, JSONSimple, JSONFlattened mean and how to choose between them.
   
   I understand you have had this discussion already and have made a decision, but I think it would be worth discussing it some more, as this decision causes our code to be more complex and the user interface more difficult to use, both meaning more work in the future.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -41,6 +48,29 @@ void reportEvent(const std::string& channel, const char* message, WORD log_level
   ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
 }
 
+struct CustomEvent {
+  std::string first;
+  std::string second;
+  std::string third;
+  int binary_length;
+  int binary_data;
+};
+
+const std::string custom_provider_name = "minifi_unit_test_provider";
+const std::string custom_channel = custom_provider_name + "/Log";
+
+bool dispatchCustomEvent(const CustomEvent& event) {
+  std::string command = "powershell \"New-WinEvent -ProviderName " + custom_provider_name

Review comment:
       changed the code to use the generated API




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,29 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;

Review comment:
       added initializer




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -43,9 +43,14 @@ namespace minifi {
 namespace processors {
 
 struct EventRender {
-  std::map<std::string, std::string> matched_fields_;
-  std::string text_;
-  std::string rendered_text_;
+  std::map<std::string, std::string> matched_fields;
+  std::string xml;
+  std::string plaintext;
+  struct {
+    std::string raw;
+    std::string simple;
+    std::string flattened;
+  } json;

Review comment:
       based on the recently reopened #618 in the future we will be able to specify multiple values for various properties, e.g. in this case `Output Format` will be able to take a list of formats, (there is already some hack with `Both` for this purpose), so the current solution takes that direction into consideration




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -692,6 +734,42 @@ void ConsumeWindowsEventLog::putEventRenderFlowFileToSession(const EventRender&
     session.getProvenanceReporter()->receive(flowFile, provenanceUri_, getUUIDStr(), "Consume windows event logs", 0);
     session.transfer(flowFile, Success);
   }
+
+  if (output_.json.raw) {
+    auto flowFile = session.create();
+    logger_->log_trace("Writing rendered raw JSON to a flow file");
+
+    session.write(flowFile, &WriteCallback(eventRender.json.raw));
+    session.putAttribute(flowFile, core::SpecialFlowAttribute::MIME_TYPE, "application/json");
+    session.putAttribute(flowFile, "Timezone name", timezone_name_);
+    session.putAttribute(flowFile, "Timezone offset", timezone_offset_);
+    session.getProvenanceReporter()->receive(flowFile, provenanceUri_, getUUIDStr(), "Consume windows event logs", 0);
+    session.transfer(flowFile, Success);

Review comment:
       factored out common code into a lambda




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,170 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+namespace {
+
+rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");
+      auto& correlation = flatten ? doc : system.AddMember("Correlation", rapidjson::kObjectType, doc.GetAllocator())["Correlation"];
+      correlation.AddMember("ActivityID", rapidjson::StringRef(correlation_xml.attribute("ActivityID").value()), doc.GetAllocator());
+    }
+
+    {
+      auto execution_xml = system_xml.child("Execution");
+      auto& execution = flatten ? doc : system.AddMember("Execution", rapidjson::kObjectType, doc.GetAllocator())["Execution"];
+      execution.AddMember("ProcessID", rapidjson::StringRef(execution_xml.attribute("ProcessID").value()), doc.GetAllocator());
+      execution.AddMember("ThreadID", rapidjson::StringRef(execution_xml.attribute("ThreadID").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("Channel", rapidjson::StringRef(system_xml.child("Channel").text().get()), doc.GetAllocator());
+    system.AddMember("Computer", rapidjson::StringRef(system_xml.child("Computer").text().get()), doc.GetAllocator());
+  }
+
+  {
+    auto eventData_xml = event_xml.child("EventData");
+    // create EventData subarray even if flatten requested
+    doc.AddMember("EventData", rapidjson::kArrayType, doc.GetAllocator());
+    for (const auto& data : eventData_xml.children()) {
+      auto name_attr = data.attribute("Name");
+      rapidjson::Value item(rapidjson::kObjectType);
+      item.AddMember("Name", rapidjson::StringRef(name_attr.value()), doc.GetAllocator());
+      item.AddMember("Content", rapidjson::StringRef(data.text().get()), doc.GetAllocator());
+      item.AddMember("Type", rapidjson::StringRef(data.name()), doc.GetAllocator());
+      // we need to query EventData because a reference to it wouldn't be stable, as we
+      // possibly add members to its parent which could result in reallocation
+      doc["EventData"].PushBack(item, doc.GetAllocator());
+      // check collision
+      if (flatten && !name_attr.empty() && !doc.HasMember(name_attr.value())) {
+        doc.AddMember(rapidjson::StringRef(name_attr.value()), rapidjson::StringRef(data.text().get()), doc.GetAllocator());

Review comment:
       If I understand correctly, using `rapidjson::StringRef` all over the place means that the returned json object is valid only as long as the original pugi xml object is valid and unchanged. If so, there should be a big loud warning on the function to warn the next caller about this.

##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument
+ *
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument

Review comment:
       I would also explain why:
   ```suggestion
    * !!WARNING!! The json document must not outlive the xml argument. For better performance,
    * the created json document stores references to values in the xml node. Accessing the
    * json document after the xml node has been destroyed results in undefined behavior.
   ```

##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument
+ *
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument

Review comment:
       I would also explain why:
   ```suggestion
    * !!WARNING!! The json document must not outlive the xml argument. For better performance,
    * the created json document stores references to values in the xml node. Accessing the
    * json document after the xml node has been changed or destroyed results in undefined behavior.
   ```




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       Even when we have list-type properties, we could have "XML,JSON" or "JSON,Plaintext" as the output format property. A user would understand what "JSON" is; they would not know (without asking us or the support team) what JSONRaw, JSONSimple, JSONFlattened mean and how to choose between them.
   
   I understand you have had this discussion already and have made a decision, but I think it would be worth discussing it some more, as this decision causes our code to be more complex and the user interface more difficult to use, both meaning more work in the future.
   




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

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



[GitHub] [nifi-minifi-cpp] szaszm closed pull request #976: MINIFICPP-1448 - CWEL JSON output

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


   


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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}
+ */
+rapidjson::Document toSimpleJSON(const pugi::xml_node& root);
+
+/**
+ * Flattens most of the structure, i.e. removes intermediate
+ * objects and lifts innermost string-valued keys to the root.
+ * e.g. {System: {Provider: {Name: String}}} => {Name: String}
+ * 
+ * Moreover it also flattens each named data element where the
+ * name does not conflict with already existing members 
+ * (e.g. a data with name "Guid" won't be flattened as it would
+ * overwrite the existing "Guid" field).
+ * 
+ * e.g. {EventData: [{Name: "Test", Content: "X"}]} => {Test: "X"}
+ * 
+ * In order to mitigate data loss, it preserves the EventData
+ * array in its entirety as well.
+ * (otherwise a "Guid" data would be lost)
+ */
+rapidjson::Document toFlattenedJSON(const pugi::xml_node& root);

Review comment:
       simple and flattened are required, raw is used for "maximum information preservation" and expected to be later transformed by a jolt processor




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       Even when we have list-type properties, we could have "XML,JSON" or "JSON,Plaintext" as the output format property. A user would understand what "JSON" is; they would not know (without asking us or the support team) what JSONRaw, JSONSimple, JSONFlattened mean and how to choose between them.  We could have a sensible default for the extra JSONFormat property (eg Simple), so most users don't need to know about it.
   
   I understand you have had this discussion already and have made a decision, but I think it would be worth discussing it some more, as this decision causes our code to be more complex and the user interface more difficult to use, both meaning more work in the future.
   




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       my personal preference is JSON to mean JSONRaw and a template-based json transformer processor (jolt processor in nifi), but based on simplicity and possible time-constraints that might not be viable




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/CWELCustomProviderTests.cpp
##########
@@ -0,0 +1,132 @@
+/**
+ * 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 "ConsumeWindowsEventLog.h"
+
+#include "core/ConfigurableComponent.h"
+#include "processors/LogAttribute.h"
+#include "processors/PutFile.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "utils/file/FileUtils.h"
+#include "rapidjson/document.h"
+
+#include "CWELTestUtils.h"
+
+// generated from the manifest file "custom-provider/unit-test-provider.man"
+// using the command "mc -um unit-test-provider.man"
+#include "custom-provider/unit-test-provider.h"
+
+namespace {
+
+struct CustomEventData {
+  std::wstring first;
+  std::wstring second;
+  std::wstring third;
+  int binary_length;
+  const unsigned char* binary_data;
+};
+
+const std::string CUSTOM_PROVIDER_NAME = "minifi_unit_test_provider";
+const std::string CUSTOM_CHANNEL = CUSTOM_PROVIDER_NAME + "/Log";
+
+bool dispatchCustomEvent(const CustomEventData& event) {
+  static auto provider_initialized = [] {
+    return EventRegisterminifi_unit_test_provider();
+  }();

Review comment:
       absolutely nothing :D 




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {

Review comment:
       done




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument
+ *
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument

Review comment:
       I would also explain why:
   ```suggestion
    * !!WARNING!! The json document must not outlive the xml argument. For better performance,
    * the created json document stores references to values in the xml node. Accessing the
    * json document after the xml node has been changed or destroyed results in undefined behavior.
   ```




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")

Review comment:
       the custom provider still gives empty plaintext output, which is quite baffling 




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       this idea came up during design and having discussed it with @arpadboda, the conclusion was that a single property is simpler than a separate json-specific property on the CWEL




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       I think it would be simpler to have a single `JSON` output format plus a new property, eg. `JSON Output Type` = `Simple|Flattened|Raw`.

##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,33 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;
-  bool apply_identifier_function_;
+  bool apply_identifier_function_{false};
   std::string provenanceUri_;
   std::string computerName_;
   uint64_t maxBufferSize_{};
   DWORD lastActivityTimestamp_{};
   std::mutex cache_mutex_;
   std::map<std::string, wel::WindowsEventLogHandler > providers_;
-  uint64_t batch_commit_size_;
+  uint64_t batch_commit_size_{};
+
+  struct OutputFormat {
+    void reset() {
+      *this = OutputFormat{};
+    }
+
+    bool xml{false};
+    bool plaintext{false};
+    struct {
+      explicit operator bool() const noexcept {
+        return raw || simple || flattened;
+      }
+
+      bool raw{false};
+      bool simple{false};
+      bool flattened{false};
+    } json;
+  } output_;

Review comment:
       I think this would be simpler this way:
   ```suggestion
     struct OutputFormat {
       bool xml{false};
       bool plaintext{false};
       bool json{false};
     };
     enum class JSONType { NONE, RAW, SIMPLE, FLATTENED };
     
     OutputFormat output_;
     JSONType json_type_{JSONType::NONE};
   ```
   The `reset()` function is not needed; you can write `output_ = {}` instead of `output_.reset()`.

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -41,6 +48,29 @@ void reportEvent(const std::string& channel, const char* message, WORD log_level
   ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
 }
 
+struct CustomEvent {
+  std::string first;
+  std::string second;
+  std::string third;
+  int binary_length;
+  int binary_data;
+};
+
+const std::string custom_provider_name = "minifi_unit_test_provider";
+const std::string custom_channel = custom_provider_name + "/Log";

Review comment:
       as these are constants, they should be in all caps: CUSTOM_CHANNEL etc

##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}

Review comment:
       This example is not clear without the XML input.

##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}
+ */
+rapidjson::Document toSimpleJSON(const pugi::xml_node& root);
+
+/**
+ * Flattens most of the structure, i.e. removes intermediate
+ * objects and lifts innermost string-valued keys to the root.
+ * e.g. {System: {Provider: {Name: String}}} => {Name: String}
+ * 
+ * Moreover it also flattens each named data element where the
+ * name does not conflict with already existing members 
+ * (e.g. a data with name "Guid" won't be flattened as it would
+ * overwrite the existing "Guid" field).
+ * 
+ * e.g. {EventData: [{Name: "Test", Content: "X"}]} => {Test: "X"}
+ * 
+ * In order to mitigate data loss, it preserves the EventData
+ * array in its entirety as well.
+ * (otherwise a "Guid" data would be lost)
+ */
+rapidjson::Document toFlattenedJSON(const pugi::xml_node& root);

Review comment:
       This Flattened format is hard to understand or to predict its output.  Do we need this?  Was it a requirement?

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());
 
-    test_controller.runSession(test_plan);
-  }
+  matchJSON(json, expected);
+}
 
-  test_plan->reset();
-  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+class JSONOutputController : public OutputFormatTestController {
+ public:
+  JSONOutputController(std::string format) : OutputFormatTestController(APPLICATION_CHANNEL, "*", std::move(format)) {}
 
-  {
+ protected:
+  void dispatchBookmarkEvent() override {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+  }
+  void dispatchCollectedEvent() override {
     reportEvent(APPLICATION_CHANNEL, "Event one");
+  }
+};
 
-    test_controller.runSession(test_plan);
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Simple correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Simple"}.run();
+  verifyJSON(event, R"json(
+    {
+      "System": {
+        "Provider": {
+          "Name": "Application"
+        },
+        "Channel": "Application"
+      },
+      "EventData": [{
+          "Type": "Data",
+          "Content": "Event one",
+          "Name": ""
+      }]
+    }
+  )json");
+}
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Raw correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Raw"}.run();
+  verifyJSON(event, R"json(
+    [
+      {
+        "name": "Event",
+        "children": [
+          {"name": "System"},
+          {
+            "name": "EventData",
+            "children": [{
+              "name": "Data",
+              "text": "Event one"
+            }] 
+          }      
+        ]
+      }
+    ]
+  )json");
+}
+
+class CustomProviderController : public OutputFormatTestController {
+ public:
+  CustomProviderController(std::string format) : OutputFormatTestController(custom_channel, "*", std::move(format)) {}
 
-    REQUIRE(LogTestController::getInstance().contains(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<EventID Qualifiers="0">14985</EventID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Level>4</Level>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Task>0</Task>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")"));
-    // the timestamp (when the event was published) goes here
-    REQUIRE(LogTestController::getInstance().contains(R"("/><EventRecordID>)"));
-    // the ID of the event goes here (a number)
-    REQUIRE(LogTestController::getInstance().contains(R"(</EventRecordID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Channel>Application</Channel><Computer>)"));
-    // the computer name goes here
-    REQUIRE(LogTestController::getInstance().contains(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)"));
+ protected:
+  void dispatchBookmarkEvent() override {
+    REQUIRE(dispatchCustomEvent({"Bookmark", "Second", "Third", 3, 12}));
+    std::this_thread::sleep_for(std::chrono::seconds{1});

Review comment:
       Why do we need a sleep here?

##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -252,11 +251,27 @@ void ConsumeWindowsEventLog::onSchedule(const std::shared_ptr<core::ProcessConte
   std::string mode;
   context->getProperty(OutputFormat.getName(), mode);
 
-  writeXML_ = (mode == Both || mode == XML);
-
-  writePlainText_ = (mode == Both || mode == Plaintext);
+  output_.reset();
+  if (mode == XML) {
+    output_.xml = true;
+  } else if (mode == Plaintext) {
+    output_.plaintext = true;
+  } else if (mode == Both) {
+    output_.xml = true;
+    output_.plaintext = true;
+  } else if (mode == JSONRaw) {
+    output_.json.raw = true;
+  } else if (mode == JSONSimple) {
+    output_.json.simple = true;
+  } else if (mode == JSONFlattened) {
+    output_.json.flattened = true;
+  } else {
+    // in the future this might be considered an error, but for now due to backwards
+    // compatibility we just fall through and execute the processor outputing nothing
+    // throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Unrecognized output format: " + mode);

Review comment:
       This would be a good candidate to revisit when we prepare version 1.0, or any major version after that.  Can we create some kind of standard marker that we can grep for? (or use it if we already have one)

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());
 
-    test_controller.runSession(test_plan);
-  }
+  matchJSON(json, expected);
+}
 
-  test_plan->reset();
-  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+class JSONOutputController : public OutputFormatTestController {
+ public:
+  JSONOutputController(std::string format) : OutputFormatTestController(APPLICATION_CHANNEL, "*", std::move(format)) {}
 
-  {
+ protected:
+  void dispatchBookmarkEvent() override {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+  }
+  void dispatchCollectedEvent() override {
     reportEvent(APPLICATION_CHANNEL, "Event one");
+  }
+};
 
-    test_controller.runSession(test_plan);
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Simple correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Simple"}.run();
+  verifyJSON(event, R"json(
+    {
+      "System": {
+        "Provider": {
+          "Name": "Application"
+        },
+        "Channel": "Application"
+      },
+      "EventData": [{
+          "Type": "Data",
+          "Content": "Event one",
+          "Name": ""
+      }]
+    }
+  )json");
+}
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Raw correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Raw"}.run();
+  verifyJSON(event, R"json(
+    [
+      {
+        "name": "Event",
+        "children": [
+          {"name": "System"},
+          {
+            "name": "EventData",
+            "children": [{
+              "name": "Data",
+              "text": "Event one"
+            }] 
+          }      
+        ]
+      }
+    ]
+  )json");
+}
+

Review comment:
       It might be worth having a `JSON::Flattened` test here, too, just to document the three types in one place.

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());

Review comment:
       very minor, but `REQUIRE_FALSE` may be more readable

##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -43,9 +43,14 @@ namespace minifi {
 namespace processors {
 
 struct EventRender {
-  std::map<std::string, std::string> matched_fields_;
-  std::string text_;
-  std::string rendered_text_;
+  std::map<std::string, std::string> matched_fields;
+  std::string xml;
+  std::string plaintext;
+  struct {
+    std::string raw;
+    std::string simple;
+    std::string flattened;
+  } json;

Review comment:
       Only one of these three can be non-empty, right?  If we had `std::string json` here, that would simplify the code in `putEventRenderFlowFileToSession()`, too.

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")

Review comment:
       Does your new manifest-based event publisher produce some output with the Plaintext output format?  If it does, then this unit test can be implemented now.

##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+static rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toRawJSON(const pugi::xml_node& root) {
+  rapidjson::Document doc;
+  if (root.type() == pugi::xml_node_type::node_document) {
+    static_cast<rapidjson::Value&>(doc) = xmlDocumentToJSON(root, doc);
+  }
+  return doc;
+}
+
+static rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");

Review comment:
       Does every event log XML have all these elements?  What happens if one of them is missing?

##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {

Review comment:
       We usually use anonymous namespaces; do you have a reason to use `static` here?

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -41,6 +48,29 @@ void reportEvent(const std::string& channel, const char* message, WORD log_level
   ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
 }
 
+struct CustomEvent {
+  std::string first;
+  std::string second;
+  std::string third;
+  int binary_length;
+  int binary_data;
+};
+
+const std::string custom_provider_name = "minifi_unit_test_provider";
+const std::string custom_channel = custom_provider_name + "/Log";
+
+bool dispatchCustomEvent(const CustomEvent& event) {
+  std::string command = "powershell \"New-WinEvent -ProviderName " + custom_provider_name

Review comment:
       I think it would be nicer to use the event writer API (https://docs.microsoft.com/en-us/windows/win32/etw/writing-manifest-based-events) instead of PowerShell.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,33 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;
-  bool apply_identifier_function_;
+  bool apply_identifier_function_{false};
   std::string provenanceUri_;
   std::string computerName_;
   uint64_t maxBufferSize_{};
   DWORD lastActivityTimestamp_{};
   std::mutex cache_mutex_;
   std::map<std::string, wel::WindowsEventLogHandler > providers_;
-  uint64_t batch_commit_size_;
+  uint64_t batch_commit_size_{};
+
+  struct OutputFormat {
+    void reset() {
+      *this = OutputFormat{};
+    }
+
+    bool xml{false};
+    bool plaintext{false};
+    struct {
+      explicit operator bool() const noexcept {
+        return raw || simple || flattened;
+      }
+
+      bool raw{false};
+      bool simple{false};
+      bool flattened{false};
+    } json;
+  } output_;

Review comment:
       this exact solution (well short of the names) was the first thing implemented, but based on [this comment](https://github.com/apache/nifi-minifi-cpp/pull/976#discussion_r559356325) it is not adequate to handle that situation




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}
+ */
+rapidjson::Document toSimpleJSON(const pugi::xml_node& root);
+
+/**
+ * Flattens most of the structure, i.e. removes intermediate
+ * objects and lifts innermost string-valued keys to the root.
+ * e.g. {System: {Provider: {Name: String}}} => {Name: String}
+ * 
+ * Moreover it also flattens each named data element where the
+ * name does not conflict with already existing members 
+ * (e.g. a data with name "Guid" won't be flattened as it would
+ * overwrite the existing "Guid" field).
+ * 
+ * e.g. {EventData: [{Name: "Test", Content: "X"}]} => {Test: "X"}
+ * 
+ * In order to mitigate data loss, it preserves the EventData
+ * array in its entirety as well.
+ * (otherwise a "Guid" data would be lost)
+ */
+rapidjson::Document toFlattenedJSON(const pugi::xml_node& root);

Review comment:
       This Flattened format is hard to understand or to predict its output.  Do we need this?  Was it a requirement?
   
   In fact, it's not clear to me why we have three kinds of JSON output.  Can we pick one of them and use that?  That would simplify things a lot, both in our code, and for the users (and the support team).




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")

Review comment:
       I will check it!




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,80 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument
+ *
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * !!WARNING!! the json document must not outlive the xml argument

Review comment:
       done




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -134,8 +135,8 @@ core::Property ConsumeWindowsEventLog::OutputFormat(
   core::PropertyBuilder::createProperty("Output Format")->
   isRequired(true)->
   withDefaultValue(Both)->
-  withAllowableValues<std::string>({XML, Plaintext, Both})->
-  withDescription("Set the output format type. In case \'Both\' is selected the processor generates two flow files for every event captured")->
+  withAllowableValues<std::string>({XML, Plaintext, Both, JSONSimple, JSONFlattened, JSONRaw})->

Review comment:
       the format has been separated to JSON + flavor in a new property `JSON Format`




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.h
##########
@@ -0,0 +1,71 @@
+/**
+ * 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
+
+#undef RAPIDJSON_ASSERT
+#define RAPIDJSON_ASSERT(x) if (!(x)) throw std::logic_error("rapidjson exception");  // NOLINT
+
+#include <pugixml.hpp>
+
+#include <stdexcept>  // for std::logic_error
+#include "rapidjson/document.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+/**
+ * Converts each xml element node to a json object of
+ * the form {name: String, attributes: Object, children: Array, text: String}
+ * Aims to preserve most of the input xml structure.
+ */
+rapidjson::Document toRawJSON(const pugi::xml_node& root);
+
+/**
+ * Retains some hierarchical structure of the original xml event,
+ * e.g. {System: {Provider: {Name: String, Guid: String}}}
+ */
+rapidjson::Document toSimpleJSON(const pugi::xml_node& root);
+
+/**
+ * Flattens most of the structure, i.e. removes intermediate
+ * objects and lifts innermost string-valued keys to the root.
+ * e.g. {System: {Provider: {Name: String}}} => {Name: String}
+ * 
+ * Moreover it also flattens each named data element where the
+ * name does not conflict with already existing members 
+ * (e.g. a data with name "Guid" won't be flattened as it would
+ * overwrite the existing "Guid" field).
+ * 
+ * e.g. {EventData: [{Name: "Test", Content: "X"}]} => {Test: "X"}
+ * 
+ * In order to mitigate data loss, it preserves the EventData
+ * array in its entirety as well.
+ * (otherwise a "Guid" data would be lost)
+ */
+rapidjson::Document toFlattenedJSON(const pugi::xml_node& root);

Review comment:
       :+1: 




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -43,9 +43,14 @@ namespace minifi {
 namespace processors {
 
 struct EventRender {
-  std::map<std::string, std::string> matched_fields_;
-  std::string text_;
-  std::string rendered_text_;
+  std::map<std::string, std::string> matched_fields;
+  std::string xml;
+  std::string plaintext;
+  struct {
+    std::string raw;
+    std::string simple;
+    std::string flattened;
+  } json;

Review comment:
       I find it hard to imagine a use case when the user would want to output two different JSON formats at the same time, but OK.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/wel/JSONUtils.cpp
##########
@@ -0,0 +1,166 @@
+/**
+ * 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 "JSONUtils.h"
+
+#include <vector>
+#include <string>
+#include <functional>
+
+#include <pugixml.hpp>
+
+#include "rapidjson/document.h"
+#include "rapidjson/writer.h"
+#include "rapidjson/stringbuffer.h"
+#include "rapidjson/prettywriter.h"
+
+#include "gsl/gsl-lite.hpp";
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace wel {
+
+static rapidjson::Value xmlElementToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_element);
+  rapidjson::Value object(rapidjson::kObjectType);
+  object.AddMember("name", rapidjson::StringRef(node.name()), doc.GetAllocator());
+  auto& attributes = object.AddMember("attributes", rapidjson::kObjectType, doc.GetAllocator())["attributes"];
+  for (const auto& attr : node.attributes()) {
+    attributes.AddMember(rapidjson::StringRef(attr.name()), rapidjson::StringRef(attr.value()), doc.GetAllocator());
+  }
+  auto& children = object.AddMember("children", rapidjson::kArrayType, doc.GetAllocator())["children"];
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  object.AddMember("text", rapidjson::StringRef(node.text().get()), doc.GetAllocator());
+  return object;
+}
+
+static rapidjson::Value xmlDocumentToJSON(const pugi::xml_node& node, rapidjson::Document& doc) {
+  gsl_Expects(node.type() == pugi::xml_node_type::node_document);
+  rapidjson::Value children(rapidjson::kArrayType);
+  for (const auto& child : node.children()) {
+    if (child.type() == pugi::xml_node_type::node_element) {
+      children.PushBack(xmlElementToJSON(child, doc), doc.GetAllocator());
+    }
+  }
+  return children;
+}
+
+rapidjson::Document toRawJSON(const pugi::xml_node& root) {
+  rapidjson::Document doc;
+  if (root.type() == pugi::xml_node_type::node_document) {
+    static_cast<rapidjson::Value&>(doc) = xmlDocumentToJSON(root, doc);
+  }
+  return doc;
+}
+
+static rapidjson::Document toJSONImpl(const pugi::xml_node& root, bool flatten) {
+  rapidjson::Document doc{rapidjson::kObjectType};
+
+  auto event_xml = root.child("Event");
+
+  {
+    auto system_xml = event_xml.child("System");
+    auto& system = flatten ? doc : doc.AddMember("System", rapidjson::kObjectType, doc.GetAllocator())["System"];
+
+    {
+      auto provider_xml = system_xml.child("Provider");
+      auto& provider = flatten ? doc : system.AddMember("Provider", rapidjson::kObjectType, doc.GetAllocator())["Provider"];
+      provider.AddMember("Name", rapidjson::StringRef(provider_xml.attribute("Name").value()), doc.GetAllocator());
+      provider.AddMember("Guid", rapidjson::StringRef(provider_xml.attribute("Guid").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventID", rapidjson::StringRef(system_xml.child("EventID").text().get()), doc.GetAllocator());
+    system.AddMember("Version", rapidjson::StringRef(system_xml.child("Version").text().get()), doc.GetAllocator());
+    system.AddMember("Level", rapidjson::StringRef(system_xml.child("Level").text().get()), doc.GetAllocator());
+    system.AddMember("Task", rapidjson::StringRef(system_xml.child("Task").text().get()), doc.GetAllocator());
+    system.AddMember("Opcode", rapidjson::StringRef(system_xml.child("Opcode").text().get()), doc.GetAllocator());
+    system.AddMember("Keywords", rapidjson::StringRef(system_xml.child("Keywords").text().get()), doc.GetAllocator());
+
+    {
+      auto timeCreated_xml = system_xml.child("TimeCreated");
+      auto& timeCreated = flatten ? doc : system.AddMember("TimeCreated", rapidjson::kObjectType, doc.GetAllocator())["TimeCreated"];
+      timeCreated.AddMember("SystemTime", rapidjson::StringRef(timeCreated_xml.attribute("SystemTime").value()), doc.GetAllocator());
+    }
+
+    system.AddMember("EventRecordID", rapidjson::StringRef(system_xml.child("EventRecordID").text().get()), doc.GetAllocator());
+
+    {
+      auto correlation_xml = system_xml.child("Correlation");

Review comment:
       in pugixml both xml_node and xml_attribute are implicitly nullable, and calling the various methods on the null versions produces sensible "null" output, e.g. `xml_node().text()` is the 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.

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -300,53 +330,249 @@ TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_fo
   outputFormatSetterTestHelper("InvalidValue", 0);
 }
 
+namespace {
+
+class OutputFormatTestController : public TestController {
+ public:
+  OutputFormatTestController(std::string channel, std::string query, std::string output_format)
+    : channel_(std::move(channel)),
+      query_(std::move(query)),
+      output_format_(std::move(output_format)) {}
+
+  std::string run() {
+    LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+    LogTestController::getInstance().setDebug<LogAttribute>();
+    std::shared_ptr<TestPlan> test_plan = createPlan();
+
+    auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), channel_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), query_);
+    test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format_);
+
+    auto dir = utils::createTempDir(this);
+
+    auto put_file = test_plan->addProcessor("PutFile", "putFile", Success, true);
+    test_plan->setProperty(put_file, PutFile::Directory.getName(), dir);
+
+    {
+      dispatchBookmarkEvent();
+
+      runSession(test_plan);
+    }
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+
+    {
+      dispatchCollectedEvent();
+
+      runSession(test_plan);
+
+      auto files = utils::file::list_dir_all(dir, LogTestController::getInstance().getLogger<LogTestController>(), false);
+      REQUIRE(files.size() == 1);
+
+      std::ifstream file{utils::file::concat_path(files[0].first, files[0].second)};
+      return {std::istreambuf_iterator<char>{file}, {}};
+    }
+  }
+
+ protected:
+  virtual void dispatchBookmarkEvent() = 0;
+  virtual void dispatchCollectedEvent() = 0;
+
+  std::string channel_;
+  std::string query_;
+  std::string output_format_;
+};
+
+}  // namespace
+
 // NOTE(fgerlits): I don't know how to unit test this, as my manually published events all result in an empty string if OutputFormat is Plaintext
 //                 but it does seem to work, based on manual tests reading system logs
 // TEST_CASE("ConsumeWindowsEventLog prints events in plain text correctly", "[onTrigger]")
 
 TEST_CASE("ConsumeWindowsEventLog prints events in XML correctly", "[onTrigger]") {
-  TestController test_controller;
-  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
-  LogTestController::getInstance().setDebug<LogAttribute>();
-  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+  class XMLFormat : public OutputFormatTestController {
+   public:
+    XMLFormat() : OutputFormatTestController(APPLICATION_CHANNEL, QUERY, "XML") {}
 
-  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Query.getName(), QUERY);
-  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+   protected:
+    void dispatchBookmarkEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+    }
+    void dispatchCollectedEvent() override {
+      reportEvent(APPLICATION_CHANNEL, "Event one");
+    }
+  } test_controller;
+
+  std::string event = test_controller.run();
+
+  REQUIRE(event.find(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)") != std::string::npos);
+  REQUIRE(event.find(R"(<EventID Qualifiers="0">14985</EventID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Level>4</Level>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Task>0</Task>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")") != std::string::npos);
+  // the timestamp (when the event was published) goes here
+  REQUIRE(event.find(R"("/><EventRecordID>)") != std::string::npos);
+  // the ID of the event goes here (a number)
+  REQUIRE(event.find(R"(</EventRecordID>)") != std::string::npos);
+  REQUIRE(event.find(R"(<Channel>Application</Channel><Computer>)") != std::string::npos);
+  // the computer name goes here
+  REQUIRE(event.find(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)") != std::string::npos);
+}
 
-  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
-  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
-  test_plan->setProperty(logger_processor, LogAttribute::LogPayload.getName(), "true");
-  test_plan->setProperty(logger_processor, LogAttribute::MaxPayloadLineLength.getName(), "1024");
+namespace {
+// carries out a loose match on objects, i.e. it doesn't matter if the
+// actual object has extra fields than expected
+void matchJSON(const rapidjson::Value& json, const rapidjson::Value& expected) {
+  if (expected.IsObject()) {
+    REQUIRE(json.IsObject());
+    for (const auto& expected_member : expected.GetObject()) {
+      REQUIRE(json.HasMember(expected_member.name));
+      matchJSON(json[expected_member.name], expected_member.value);
+    }
+  } else if (expected.IsArray()) {
+    REQUIRE(json.IsArray());
+    REQUIRE(json.Size() == expected.Size());
+    for (size_t idx{0}; idx < expected.Size(); ++idx) {
+      matchJSON(json[idx], expected[idx]);
+    }
+  } else {
+    REQUIRE(json == expected);
+  }
+}
 
-  {
-    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+void verifyJSON(const std::string& json_str, const std::string& expected_str) {
+  rapidjson::Document json, expected;
+  REQUIRE(!json.Parse(json_str.c_str()).HasParseError());
+  REQUIRE(!expected.Parse(expected_str.c_str()).HasParseError());
 
-    test_controller.runSession(test_plan);
-  }
+  matchJSON(json, expected);
+}
 
-  test_plan->reset();
-  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+class JSONOutputController : public OutputFormatTestController {
+ public:
+  JSONOutputController(std::string format) : OutputFormatTestController(APPLICATION_CHANNEL, "*", std::move(format)) {}
 
-  {
+ protected:
+  void dispatchBookmarkEvent() override {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+  }
+  void dispatchCollectedEvent() override {
     reportEvent(APPLICATION_CHANNEL, "Event one");
+  }
+};
 
-    test_controller.runSession(test_plan);
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Simple correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Simple"}.run();
+  verifyJSON(event, R"json(
+    {
+      "System": {
+        "Provider": {
+          "Name": "Application"
+        },
+        "Channel": "Application"
+      },
+      "EventData": [{
+          "Type": "Data",
+          "Content": "Event one",
+          "Name": ""
+      }]
+    }
+  )json");
+}
+
+TEST_CASE("ConsumeWindowsEventLog prints events in JSON::Raw correctly", "[onTrigger]") {
+  std::string event = JSONOutputController{"JSON::Raw"}.run();
+  verifyJSON(event, R"json(
+    [
+      {
+        "name": "Event",
+        "children": [
+          {"name": "System"},
+          {
+            "name": "EventData",
+            "children": [{
+              "name": "Data",
+              "text": "Event one"
+            }] 
+          }      
+        ]
+      }
+    ]
+  )json");
+}
+
+class CustomProviderController : public OutputFormatTestController {
+ public:
+  CustomProviderController(std::string format) : OutputFormatTestController(custom_channel, "*", std::move(format)) {}
 
-    REQUIRE(LogTestController::getInstance().contains(R"(<Event xmlns="http://schemas.microsoft.com/win/2004/08/events/event"><System><Provider Name="Application"/>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<EventID Qualifiers="0">14985</EventID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Level>4</Level>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Task>0</Task>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Keywords>0x80000000000000</Keywords><TimeCreated SystemTime=")"));
-    // the timestamp (when the event was published) goes here
-    REQUIRE(LogTestController::getInstance().contains(R"("/><EventRecordID>)"));
-    // the ID of the event goes here (a number)
-    REQUIRE(LogTestController::getInstance().contains(R"(</EventRecordID>)"));
-    REQUIRE(LogTestController::getInstance().contains(R"(<Channel>Application</Channel><Computer>)"));
-    // the computer name goes here
-    REQUIRE(LogTestController::getInstance().contains(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)"));
+ protected:
+  void dispatchBookmarkEvent() override {
+    REQUIRE(dispatchCustomEvent({"Bookmark", "Second", "Third", 3, 12}));
+    std::this_thread::sleep_for(std::chrono::seconds{1});

Review comment:
       it is probably due to using powershell to dispatch the event, I hope using the API you linked will make it obsolete




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.cpp
##########
@@ -692,6 +734,42 @@ void ConsumeWindowsEventLog::putEventRenderFlowFileToSession(const EventRender&
     session.getProvenanceReporter()->receive(flowFile, provenanceUri_, getUUIDStr(), "Consume windows event logs", 0);
     session.transfer(flowFile, Success);
   }
+
+  if (output_.json.raw) {
+    auto flowFile = session.create();
+    logger_->log_trace("Writing rendered raw JSON to a flow file");
+
+    session.write(flowFile, &WriteCallback(eventRender.json.raw));
+    session.putAttribute(flowFile, core::SpecialFlowAttribute::MIME_TYPE, "application/json");
+    session.putAttribute(flowFile, "Timezone name", timezone_name_);
+    session.putAttribute(flowFile, "Timezone offset", timezone_offset_);
+    session.getProvenanceReporter()->receive(flowFile, provenanceUri_, getUUIDStr(), "Consume windows event logs", 0);
+    session.transfer(flowFile, Success);

Review comment:
       These lines seem to be the same for all json output formats. We could a single branch for all json output formats and only distinguish the `session.write` calls by json format type.

##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,29 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;

Review comment:
       I suppose this should be initialized as well.




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/tests/CWELCustomProviderTests.cpp
##########
@@ -0,0 +1,132 @@
+/**
+ * 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 "ConsumeWindowsEventLog.h"
+
+#include "core/ConfigurableComponent.h"
+#include "processors/LogAttribute.h"
+#include "processors/PutFile.h"
+#include "TestBase.h"
+#include "utils/TestUtils.h"
+#include "utils/file/FileUtils.h"
+#include "rapidjson/document.h"
+
+#include "CWELTestUtils.h"
+
+// generated from the manifest file "custom-provider/unit-test-provider.man"
+// using the command "mc -um unit-test-provider.man"
+#include "custom-provider/unit-test-provider.h"
+
+namespace {
+
+struct CustomEventData {
+  std::wstring first;
+  std::wstring second;
+  std::wstring third;
+  int binary_length;
+  const unsigned char* binary_data;
+};
+
+const std::string CUSTOM_PROVIDER_NAME = "minifi_unit_test_provider";
+const std::string CUSTOM_CHANNEL = CUSTOM_PROVIDER_NAME + "/Log";
+
+bool dispatchCustomEvent(const CustomEventData& event) {
+  static auto provider_initialized = [] {
+    return EventRegisterminifi_unit_test_provider();
+  }();

Review comment:
       what is the advantage of this over
   ```
   static auto provider_initialized = EventRegisterminifi_unit_test_provider();
   ```
   ?




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

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 #976: MINIFICPP-1448 - CWEL JSON output

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



##########
File path: extensions/windows-event-log/ConsumeWindowsEventLog.h
##########
@@ -133,17 +141,29 @@ class ConsumeWindowsEventLog : public core::Processor {
   std::wstring wstrQuery_;
   std::string regex_;
   bool resolve_as_attributes_;

Review comment:
       initialized




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

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