This is an automated email from the ASF dual-hosted git repository. markap14 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git
commit ce5eae5b2cd5cf49a90d2980c58cd6647f4af292 Author: Mark Payne <[email protected]> AuthorDate: Tue Oct 29 10:17:27 2019 -0400 NIFI-6800: Added unit test to verify behavior of contribution --- .../repository/TestStandardProcessSession.java | 48 +++++++++++++++++++++- 1 file changed, 46 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index b2e1870..267f22d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -19,6 +19,7 @@ package org.apache.nifi.controller.repository; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.Counter; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.NopConnectionEventListener; @@ -112,6 +113,7 @@ public class TestStandardProcessSession { private ProvenanceEventRepository provenanceRepo; private MockFlowFileRepository flowFileRepo; + private CounterRepository counterRepository; private final Relationship FAKE_RELATIONSHIP = new Relationship.Builder().name("FAKE").build(); private static StandardResourceClaimManager resourceClaimManager; @@ -154,7 +156,7 @@ public class TestStandardProcessSession { System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardProcessSession.class.getResource("/conf/nifi.properties").getFile()); final FlowFileEventRepository flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class); - final CounterRepository counterRepo = Mockito.mock(CounterRepository.class); + counterRepository = new StandardCounterRepository(); provenanceRepo = new MockProvenanceRepository(); final Connection connection = createConnection(); @@ -194,7 +196,7 @@ public class TestStandardProcessSession { contentRepo.initialize(new StandardResourceClaimManager()); flowFileRepo = new MockFlowFileRepository(contentRepo); - context = new RepositoryContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo); + context = new RepositoryContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepository, provenanceRepo); session = new StandardProcessSession(context, () -> false); } @@ -297,6 +299,48 @@ public class TestStandardProcessSession { } @Test + public void testCheckpointMergesCounters() { + final Relationship relationship = new Relationship.Builder().name("A").build(); + + FlowFile flowFile = session.create(); + session.transfer(flowFile, relationship); + session.adjustCounter("a", 1, false); + session.checkpoint(); + + flowFile = session.create(); + session.transfer(flowFile, relationship); + session.adjustCounter("a", 1, false); + session.adjustCounter("b", 3, false); + session.checkpoint(); + + assertEquals(0, counterRepository.getCounters().size()); + session.commit(); + + // We should have 2 different counters with the name "a" and 2 different counters with the name "b" - + // one for the "All Instances" context and one for the individual instance's context. + final List<Counter> counters = counterRepository.getCounters(); + assertEquals(4, counters.size()); + + int aCounters = 0; + int bCounters = 0; + for (final Counter counter : counters) { + switch (counter.getName()) { + case "a": + assertEquals(2, counter.getValue()); + aCounters++; + break; + case "b": + assertEquals(3, counter.getValue()); + bCounters++; + break; + } + } + + assertEquals(2, aCounters); + assertEquals(2, bCounters); + } + + @Test public void testHandlingOfMultipleFlowFilesWithSameId() { // Add two FlowFiles with the same ID for (int i=0; i < 2; i++) {
