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/12 09:32:10 UTC

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

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