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 2020/07/09 15:56:46 UTC

[GitHub] [nifi-minifi-cpp] fgerlits opened a new pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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


   Unit tests for the ConsumeWindowsEventLog processor and the related helper classes Bookmark and MetadataWalker.
   
   Small changes in the classes under test to help unit testing:
   * move the unique_evt_handle class from Bookmark.h to its own header, so we can reuse it,
   * refactor MetadataWalker to make the Windows event dependencies mockable.
   
   Coverage is far from perfect.  The two main gaps are:
   * the processor is not tested in `ProcessOldEvents=true` mode, as this would read the event log from the start and there is no way (that I know of) to clear the event log without Administrator rights,
   * the Plaintext output format is not tested, only the XML format, as (I think) one needs to publish manifest-based events for `EvtFormatMessage(... flags=EvtFormatMessageEvent ...)` to work, and registering a manifest requires Administrator rights, too.
   
   I have tested these use cases manually, but clearly a unit test would be better.  If you have an idea how to do that, please let me know.
   
   ---
   
   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:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [x] 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.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
   
   - [ ] 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 travis-ci 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] fgerlits commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);

Review comment:
       It is slightly different, because the default value of `uuid` is `utils::Identifier()`, which is blank/all-0s.  I wanted to exercise both code paths, but it's probably not really necessary.  Do you want me to remove 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] fgerlits commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;
+  DWORD num_results_found = 0;
+  bool result = EvtNext(results.get(), 1, &event_raw_handle, timeout_ms, 0, &num_results_found);

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] arpadboda commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);

Review comment:
       Nope, I was just interested in the added value. This covers another path, it's fine, thanks!




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

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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);

Review comment:
       https://issues.apache.org/jira/browse/MINIFICPP-1292




----------------------------------------------------------------
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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;
+  DWORD num_results_found = 0;
+  bool result = EvtNext(results.get(), 1, &event_raw_handle, timeout_ms, 0, &num_results_found);

Review comment:
       we check it in the next line; would you prefer to inline the `EvtNext` call into `if (result)`?  (or maybe rename `result` to `success`?)




----------------------------------------------------------------
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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;

Review comment:
       done

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"

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] arpadboda commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"

Review comment:
       You can avoid this by adding standard processors to the cmake file of the test as "target_include"dir"

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);

Review comment:
       I think @adebreceni 's latest change with property validation should address this issue. 
   Could you guys sync?

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);

Review comment:
       What's the point in "manually" generating the uuid here?
   As far as I remember, it's done by default in case it's not specified. 

##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;

Review comment:
       Could you define this at the beginning of the file? 
   I prefer to have constants there with capital case. 

##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;
+  DWORD num_results_found = 0;
+  bool result = EvtNext(results.get(), 1, &event_raw_handle, timeout_ms, 0, &num_results_found);

Review comment:
       Nitpicking: this variable seems to be needless. 

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,

Review comment:
       This one is obsolete as well. 
   This was introduced back in those days when we had no central state storage for processor, but the current implementation of CWEL doesn't rely on this any more as it uses the corresponding controller service to store state. 

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);
+}
+
+// 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

Review comment:
       Hm, this is weird as we seem to successfully capture event data as we can render it in xml format.
   I would definitely be interested in the root cause of this. 

##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);
+}
+
+// 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();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+
+  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");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    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><Level>4</Level><Task>0</Task><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><Channel>Application</Channel><Computer>)"));
+    // the computer name goes here
+    REQUIRE(LogTestController::getInstance().contains(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)"));
+  }
+}
+
+namespace {
+
+int countOccurrences(const std::string& text, const std::string& substring) {

Review comment:
       There is a function called ```countPatInStr``` in IntegrationBase.h doing quite the same.
   
   Although I feel like this logic should be moved to LogTestController, so both unit and integration tests could depend on it. 




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

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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);
+}
+
+// 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();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), "XML");
+
+  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");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    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><Level>4</Level><Task>0</Task><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><Channel>Application</Channel><Computer>)"));
+    // the computer name goes here
+    REQUIRE(LogTestController::getInstance().contains(R"(</Computer><Security/></System><EventData><Data>Event one</Data></EventData></Event>)"));
+  }
+}
+
+namespace {
+
+int countOccurrences(const std::string& text, const std::string& substring) {

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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,

Review comment:
       I've added a comment about deprecation




----------------------------------------------------------------
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] arpadboda commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);
+}
+
+// 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

Review comment:
       That makes sense, thanks! 




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

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



[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);

Review comment:
       Yep, a follow-up would make sense to me. Not sure we can introduce this without a breaking change, so might be targeted for 1.0, but definitely seems to be a useful improvement to me.




----------------------------------------------------------------
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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);

Review comment:
       He has told me his change was for numeric (and I think bool) properties only; `withAllowableValues()` does not throw on a non-allowable value, it only includes the list of allowable values in the heartbeat so clients can do valuation on their side.  Shall I create a Jira to add validation?




----------------------------------------------------------------
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] arpadboda commented on pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

Posted by GitBox <gi...@apache.org>.
arpadboda commented on pull request #836:
URL: https://github.com/apache/nifi-minifi-cpp/pull/836#issuecomment-658855469


   @fgerlits : 
   The stringutils part seems to fail on gcc:
   https://travis-ci.org/github/apache/nifi-minifi-cpp/jobs/708347373


----------------------------------------------------------------
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 #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/ConsumeWindowsEventLogTests.cpp
##########
@@ -0,0 +1,393 @@
+/**
+ * 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 "../../../../extensions/standard-processors/processors/LogAttribute.h"
+#include "TestBase.h"
+
+using ConsumeWindowsEventLog = org::apache::nifi::minifi::processors::ConsumeWindowsEventLog;
+using LogAttribute = org::apache::nifi::minifi::processors::LogAttribute;
+using ConfigurableComponent = org::apache::nifi::minifi::core::ConfigurableComponent;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+core::Relationship Success{"success", "Everything is fine"};
+
+const std::string APPLICATION_CHANNEL = "Application";
+
+constexpr DWORD CWEL_TESTS_OPCODE = 14985;  // random opcode hopefully won't clash with something important
+
+void reportEvent(const std::string& channel, const char* message, WORD log_level = EVENTLOG_INFORMATION_TYPE) {
+  auto event_source = RegisterEventSourceA(nullptr, channel.c_str());
+  auto deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, log_level, 0, CWEL_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog constructor works", "[create]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  REQUIRE_NOTHROW(ConsumeWindowsEventLog processor_one("one"));
+
+  REQUIRE_NOTHROW(
+    utils::Identifier uuid = utils::IdGenerator::getIdGenerator()->generate();
+    ConsumeWindowsEventLog processor_two("two", uuid);
+  );  // NOLINT
+
+  REQUIRE_NOTHROW(
+    auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  );  // NOLINT
+}
+
+TEST_CASE("ConsumeWindowsEventLog properties work with default values", "[create][properties]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConfigurableComponent>();
+  LogTestController::getInstance().setTrace<ConsumeWindowsEventLog>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_controller.runSession(test_plan);
+
+  auto properties_required_or_with_default_value = {
+    ConsumeWindowsEventLog::Channel,
+    ConsumeWindowsEventLog::Query,
+    // ConsumeWindowsEventLog::RenderFormatXML,  // FIXME(fgerlits): not defined, does not exist in NiFi either; should be removed
+    ConsumeWindowsEventLog::MaxBufferSize,
+    // ConsumeWindowsEventLog::InactiveDurationToReconnect,  // FIXME(fgerlits): obsolete, see definition; should be removed
+    ConsumeWindowsEventLog::IdentifierMatcher,
+    ConsumeWindowsEventLog::IdentifierFunction,
+    ConsumeWindowsEventLog::ResolveAsAttributes,
+    ConsumeWindowsEventLog::EventHeader,
+    ConsumeWindowsEventLog::OutputFormat,
+    ConsumeWindowsEventLog::BatchCommitSize,
+    ConsumeWindowsEventLog::BookmarkRootDirectory,
+    ConsumeWindowsEventLog::ProcessOldEvents
+  };
+  for (const core::Property& property : properties_required_or_with_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + " value ")) {
+      FAIL("Property did not get queried: " << property.getName());
+    }
+  }
+
+  auto properties_optional_without_default_value = {
+    ConsumeWindowsEventLog::EventHeaderDelimiter
+  };
+  for (const core::Property& property : properties_optional_without_default_value) {
+    if (!LogTestController::getInstance().contains("property name " + property.getName() + ", empty value")) {
+      FAIL("Optional property did not get queried: " << property.getName());
+    }
+  }
+
+  REQUIRE(LogTestController::getInstance().contains("Successfully configured CWEL"));
+}
+
+TEST_CASE("ConsumeWindowsEventLog onSchedule throws if it cannot create the bookmark", "[create][bookmark]") {
+  TestController test_controller;
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(processor, ConsumeWindowsEventLog::Channel.getName(), "NonexistentChannel1234981");
+
+  REQUIRE_THROWS_AS(test_controller.runSession(test_plan), minifi::Exception);
+}
+
+TEST_CASE("ConsumeWindowsEventLog can consume new events", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  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");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+  // event zero is not reported as the bookmark is created on the first run
+  // and we use the default config setting ProcessOldEvents = false
+  // later runs will start with a bookmark saved in the state manager
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read one event") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event one</Data></EventData>"));
+  }
+
+  SECTION("Read two events") {
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event two</Data></EventData>"));
+    REQUIRE(LogTestController::getInstance().contains("<EventData><Data>Event three</Data></EventData>"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog bookmarking works", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  reportEvent(APPLICATION_CHANNEL, "Event zero");
+
+  test_controller.runSession(test_plan);
+  REQUIRE(LogTestController::getInstance().contains("processed 0 Events"));
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  SECTION("Read in one go") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 3 Events"));
+  }
+
+  SECTION("Read in two batches") {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 1 Events"));
+
+    reportEvent(APPLICATION_CHANNEL, "Event two");
+    reportEvent(APPLICATION_CHANNEL, "Event three");
+
+    test_plan->reset();
+    LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+    test_controller.runSession(test_plan);
+    REQUIRE(LogTestController::getInstance().contains("processed 2 Events"));
+  }
+}
+
+TEST_CASE("ConsumeWindowsEventLog extracts some attributes by default", "[onTrigger]") {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  // 0th event, only to create a bookmark
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 1st event, on Info level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one: something interesting happened", EVENTLOG_INFORMATION_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Information"));
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  // 2st event, on Warning level
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event two: something fishy happened!", EVENTLOG_WARNING_TYPE);
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("key:Keywords value:Classic"));
+    REQUIRE(LogTestController::getInstance().contains("key:Level value:Warning"));
+  }
+}
+
+namespace {
+
+void outputFormatSetterTestHelper(const std::string &output_format, int expected_num_flow_files) {
+  TestController test_controller;
+  LogTestController::getInstance().setDebug<ConsumeWindowsEventLog>();
+  LogTestController::getInstance().setDebug<LogAttribute>();
+  std::shared_ptr<TestPlan> test_plan = test_controller.createPlan();
+
+  auto cwel_processor = test_plan->addProcessor("ConsumeWindowsEventLog", "cwel");
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::Channel.getName(), APPLICATION_CHANNEL);
+  test_plan->setProperty(cwel_processor, ConsumeWindowsEventLog::OutputFormat.getName(), output_format);
+
+  auto logger_processor = test_plan->addProcessor("LogAttribute", "logger", Success, true);
+  test_plan->setProperty(logger_processor, LogAttribute::FlowFilesToLog.getName(), "0");
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event zero: this is in the past");
+
+    test_controller.runSession(test_plan);
+  }
+
+  test_plan->reset();
+  LogTestController::getInstance().resetStream(LogTestController::getInstance().log_output);
+
+  {
+    reportEvent(APPLICATION_CHANNEL, "Event one");
+
+    test_controller.runSession(test_plan);
+
+    REQUIRE(LogTestController::getInstance().contains("Logged " + std::to_string(expected_num_flow_files) + " flow files"));
+  }
+}
+
+}  // namespace
+
+TEST_CASE("ConsumeWindowsEventLog output format can be set", "[create][output_format]") {
+  outputFormatSetterTestHelper("XML", 1);
+  outputFormatSetterTestHelper("Plaintext", 1);
+  outputFormatSetterTestHelper("Both", 2);
+
+  // NOTE(fgerlits): this may be a bug, as I would expect this to throw in onSchedule(),
+  // but it starts merrily, just does not write flow files in either format
+  outputFormatSetterTestHelper("InvalidValue", 0);
+}
+
+// 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

Review comment:
       Proper manifest-based event types have a dll containing string mappings used by `EvtFormatMessage(... flags=EvtFormatMessageEvent ...)`, so XML like
   ```xml
   <EventData>
     <Binary>(some hex-encoded binary data)</Binary>
     <Data>Cloudera Digital Delivery Services</Data>
     <Data>4.0.52.0</Data>
     <Data>1033</Data>
     <Data>Cloudera Inc</Data>
     <Data>0</Data>
   </EventData>
   ```
   can get transformed to plain text like
   ```
   Windows Installer reconfigured the product.
   Product Name: Cloudera Digital Delivery Services.
   Product Version: 4.0.52.0.
   Product Language: 1033.
   Manufacturer: Cloudera Inc.
   Reconfiguration success or error status: 0.
   ```
   I think the reason I get an empty result is that `EvtFormatMessage` is not able to find these mappings for my ad-hoc events.  I will do some more investigation; maybe it is possible to use some predefined event type which has predefined mappings.




----------------------------------------------------------------
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] arpadboda closed pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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


   


----------------------------------------------------------------
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] arpadboda commented on a change in pull request #836: MINIFICPP-1248 Create unit tests for the ConsumeWindowsEventLog processor

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



##########
File path: extensions/windows-event-log/tests/BookmarkTests.cpp
##########
@@ -0,0 +1,245 @@
+/**
+ * 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 "Bookmark.h"
+
+#include <regex>
+
+#include "TestBase.h"
+#include "utils/gsl.h"
+#include "wel/UniqueEvtHandle.h"
+
+using Bookmark = org::apache::nifi::minifi::processors::Bookmark;
+using unique_evt_handle = org::apache::nifi::minifi::wel::unique_evt_handle;
+using IdGenerator = org::apache::nifi::minifi::utils::IdGenerator;
+
+namespace {
+
+const std::wstring APPLICATION_CHANNEL = L"Application";
+
+constexpr DWORD BOOKMARK_TESTS_OPCODE = 10368;  // random opcode hopefully won't clash with something important
+
+std::unique_ptr<Bookmark> createBookmark(TestPlan &test_plan,
+                                         const std::wstring &channel,
+                                         const std::string &uuid = IdGenerator::getIdGenerator()->generate().to_string()) {
+  const auto state_manager = test_plan.getStateManagerProvider()->getCoreComponentStateManager(uuid);
+  const auto logger = test_plan.getLogger();
+  return utils::make_unique<Bookmark>(channel, L"*", "", uuid, false, state_manager, logger);
+}
+
+void reportEvent(const std::wstring& channel, const char* message) {
+  auto event_source = RegisterEventSourceW(nullptr, channel.c_str());
+  auto event_source_deleter = gsl::finally([&event_source](){ DeregisterEventSource(event_source); });
+  ReportEventA(event_source, EVENTLOG_INFORMATION_TYPE, 0,
+               BOOKMARK_TESTS_OPCODE, nullptr, 1, 0, &message, nullptr);
+}
+
+std::wstring bookmarkHandleAsXml(EVT_HANDLE event) {
+  REQUIRE(event);
+  constexpr std::size_t BUFFER_SIZE = 1024;
+  std::array<wchar_t, BUFFER_SIZE> buffer = {};
+  DWORD buffer_used;
+  DWORD property_count;
+  if (!EvtRender(nullptr, event, EvtRenderBookmark, buffer.size(), buffer.data(), &buffer_used, &property_count)) {
+    FAIL("Could not render event; error code: " << GetLastError());
+  }
+  return std::wstring{buffer.data()};
+}
+
+std::wstring bookmarkAsXml(const std::unique_ptr<Bookmark>& bookmark) {
+  REQUIRE(bookmark);
+  REQUIRE(*bookmark);
+  return bookmarkHandleAsXml(bookmark->getBookmarkHandleFromXML());
+}
+
+unique_evt_handle queryEvents() {
+  std::wstring query = L"Event/System/EventID=" + std::to_wstring(BOOKMARK_TESTS_OPCODE);
+  unique_evt_handle results{EvtQuery(NULL, APPLICATION_CHANNEL.c_str(), query.c_str(), EvtQueryChannelPath | EvtQueryReverseDirection)};
+  if (!results) {
+    FAIL("EvtQuery() failed; error code: " << GetLastError());
+  }
+  return results;
+}
+
+unique_evt_handle getFirstEventFromResults(const unique_evt_handle& results) {
+  REQUIRE(results);
+  EVT_HANDLE event_raw_handle = 0;
+  DWORD timeout_ms = 100;
+  DWORD num_results_found = 0;
+  bool result = EvtNext(results.get(), 1, &event_raw_handle, timeout_ms, 0, &num_results_found);

Review comment:
       I just meant embedding in the if statement, but I can live with the current code as well, not a big deal. 




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