You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2021/03/25 16:38:18 UTC

[nifi] branch main updated: NIFI-8364 This closes #4935. Refactored TestQuerySolr and TestGetSolr to reuse embedded SolrClient

This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 54d2222  NIFI-8364 This closes #4935. Refactored TestQuerySolr and TestGetSolr to reuse embedded SolrClient
54d2222 is described below

commit 54d2222fb0054c263049ff83653d4ba2d29e439f
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed Mar 24 15:51:01 2021 -0500

    NIFI-8364 This closes #4935. Refactored TestQuerySolr and TestGetSolr to reuse embedded SolrClient
    
    - Replaced Gson with Jackson for parsing JSON
    - Corrected assertion argument ordering
    - Simplified relative path determination for EmbeddedSolrServerFactory
    - Replaced SimpleDateFormat with java.time.Instant parsing and formatting
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../apache/nifi/processors/solr/TestGetSolr.java   | 229 ++++++-------
 .../apache/nifi/processors/solr/TestQuerySolr.java | 366 ++++++++-------------
 2 files changed, 234 insertions(+), 361 deletions(-)

diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java
index ee1c386..602126a 100644
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestGetSolr.java
@@ -18,94 +18,80 @@
  */
 package org.apache.nifi.processors.solr;
 
-import com.google.gson.stream.JsonReader;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.json.JsonRecordSetWriter;
 import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.common.SolrInputDocument;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.xmlunit.matchers.CompareMatcher;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
+import java.time.Instant;
 import java.util.Date;
-import java.util.Locale;
-import java.util.TimeZone;
+import java.util.stream.StreamSupport;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 
 public class TestGetSolr {
+    private static final String DEFAULT_SOLR_CORE = "testCollection";
 
-    static final String DEFAULT_SOLR_CORE = "testCollection";
+    private static final String DATE_STRING_EARLIER = "1970-01-01T00:00:00.000Z";
 
-    final static SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.US);
-    final static String DATE_STRING_EARLIER = "1970-01-01T00:00:00.000Z";
-    final static String DATE_STRING_LATER = "1970-01-01T00:00:00.001Z";
-    static {
-        DATE_FORMAT.setTimeZone(TimeZone.getTimeZone("GMT"));
+    private static final String DATE_STRING_LATER = "1970-01-01T00:00:00.001Z";
+
+    private static SolrClient solrClient;
+
+    @BeforeClass
+    public static void createSolrClient() throws Exception {
+        solrClient = createEmbeddedSolrClient();
     }
 
-    private SolrClient solrClient;
-
-    @Before
-    public void setup() {
-
-        try {
-
-            // create an EmbeddedSolrServer for the processor to use
-            final String relPath = getClass().getProtectionDomain().getCodeSource()
-                    .getLocation().getFile() + "../../target";
-
-            solrClient = EmbeddedSolrServerFactory.create(EmbeddedSolrServerFactory.DEFAULT_SOLR_HOME,
-                    DEFAULT_SOLR_CORE, relPath);
-
-            final Date date = DATE_FORMAT.parse(DATE_STRING_EARLIER);
-
-            for (int i = 0; i < 10; i++) {
-                SolrInputDocument doc = new SolrInputDocument();
-                doc.addField("id", "doc" + i);
-                doc.addField("created", date);
-                doc.addField("string_single", "single" + i + ".1");
-                doc.addField("string_multi", "multi" + i + ".1");
-                doc.addField("string_multi", "multi" + i + ".2");
-                doc.addField("integer_single", i);
-                doc.addField("integer_multi", 1);
-                doc.addField("integer_multi", 2);
-                doc.addField("integer_multi", 3);
-                doc.addField("double_single", 0.5 + i);
-                solrClient.add(doc);
-
-            }
-            solrClient.commit();
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail(e.getMessage());
-        }
+    @AfterClass
+    public static void closeSolrClient() throws Exception {
+        solrClient.close();
     }
 
-    @After
-    public void teardown() {
-        try {
-            solrClient.close();
-        } catch (Exception e) {
+    private static SolrClient createEmbeddedSolrClient() throws IOException, SolrServerException {
+        final String relPath = TestGetSolr.class.getResource("/").getPath();
+
+        final SolrClient embeddedSolrClient = EmbeddedSolrServerFactory.create(EmbeddedSolrServerFactory.DEFAULT_SOLR_HOME,
+                DEFAULT_SOLR_CORE, relPath);
+
+        final Date date = Date.from(Instant.parse(DATE_STRING_EARLIER));
+
+        for (int i = 0; i < 10; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", "doc" + i);
+            doc.addField("created", date);
+            doc.addField("string_single", "single" + i + ".1");
+            doc.addField("string_multi", "multi" + i + ".1");
+            doc.addField("string_multi", "multi" + i + ".2");
+            doc.addField("integer_single", i);
+            doc.addField("integer_multi", 1);
+            doc.addField("integer_multi", 2);
+            doc.addField("integer_multi", 3);
+            doc.addField("double_single", 0.5 + i);
+            embeddedSolrClient.add(doc);
+
         }
+        embeddedSolrClient.commit();
+        return embeddedSolrClient;
     }
 
     private static TestRunner createDefaultTestRunner(GetSolr processor) {
@@ -120,47 +106,43 @@ public class TestGetSolr {
     }
 
     @Test
-    public void testLessThanBatchSizeShouldProduceOneFlowFile() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testLessThanBatchSizeShouldProduceOneFlowFile() {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "20");
 
-        runner.run();
+        runner.run(1, false);
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 1);
     }
 
     @Test
-    public void testNoResultsShouldProduceNoOutput() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testNoResultsShouldProduceNoOutput() {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.SOLR_QUERY, "integer_single:1000");
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
 
-        runner.run();
+        runner.run(1, false);
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 0);
     }
 
-    @Test
-    public void testSolrModes() throws IOException, SolrServerException {
-
-    }
-
     @Test(expected = java.lang.AssertionError.class)
-    public void testValidation() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testValidation() {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "2");
         runner.setProperty(GetSolr.RETURN_TYPE, GetSolr.MODE_REC.getValue());
 
-        runner.run(1);
+        runner.run(1, false);
     }
 
     @Test
-    public void testCompletenessDespiteUpdates() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testCompletenessDespiteUpdates() throws Exception {
+        final SolrClient testSolrClient = createEmbeddedSolrClient();
+        final TestableProcessor proc = new TestableProcessor(testSolrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
@@ -177,19 +159,22 @@ public class TestGetSolr {
         doc1.addField("id", "doc1");
         doc1.addField("created", new Date());
 
-        solrClient.add(doc0);
-        solrClient.add(doc1);
-        solrClient.commit();
+        testSolrClient.add(doc0);
+        testSolrClient.add(doc1);
+        testSolrClient.commit();
 
         runner.run(1,true, false);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 2);
         runner.assertAllFlowFilesContainAttribute(CoreAttributes.MIME_TYPE.key());
+
+        testSolrClient.close();
     }
 
     @Test
     public void testCompletenessDespiteDeletions() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+        final SolrClient testSolrClient = createEmbeddedSolrClient();
+        final TestableProcessor proc = new TestableProcessor(testSolrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
@@ -206,26 +191,29 @@ public class TestGetSolr {
         doc11.addField("id", "doc11");
         doc11.addField("created", new Date());
 
-        solrClient.add(doc10);
-        solrClient.add(doc11);
-        solrClient.deleteById("doc0");
-        solrClient.deleteById("doc1");
-        solrClient.deleteById("doc2");
-        solrClient.commit();
+        testSolrClient.add(doc10);
+        testSolrClient.add(doc11);
+        testSolrClient.deleteById("doc0");
+        testSolrClient.deleteById("doc1");
+        testSolrClient.deleteById("doc2");
+        testSolrClient.commit();
 
         runner.run(1,true, false);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 2);
         runner.assertAllFlowFilesContainAttribute(CoreAttributes.MIME_TYPE.key());
+
+        testSolrClient.close();
     }
 
     @Test
-    public void testInitialDateFilter() throws IOException, SolrServerException, ParseException {
-        final Date dateToFilter = DATE_FORMAT.parse(DATE_STRING_LATER);
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testInitialDateFilter() throws IOException, SolrServerException {
+        final SolrClient testSolrClient = createEmbeddedSolrClient();
+        final Date dateToFilter = Date.from(Instant.parse(DATE_STRING_LATER));
+        final TestableProcessor proc = new TestableProcessor(testSolrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
-        runner.setProperty(GetSolr.DATE_FILTER, DATE_FORMAT.format(dateToFilter));
+        runner.setProperty(GetSolr.DATE_FILTER, DATE_STRING_LATER);
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
 
         SolrInputDocument doc10 = new SolrInputDocument();
@@ -235,19 +223,21 @@ public class TestGetSolr {
         doc11.addField("id", "doc11");
         doc11.addField("created", dateToFilter);
 
-        solrClient.add(doc10);
-        solrClient.add(doc11);
-        solrClient.commit();
+        testSolrClient.add(doc10);
+        testSolrClient.add(doc11);
+        testSolrClient.commit();
 
         runner.run(1,true, true);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 2);
         runner.assertAllFlowFilesContainAttribute(CoreAttributes.MIME_TYPE.key());
+
+        testSolrClient.close();
     }
 
     @Test
-    public void testPropertyModified() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testPropertyModified() {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
@@ -266,15 +256,15 @@ public class TestGetSolr {
 
         // Change property not contained in propertyNamesForActivatingClearState
         runner.setProperty(GetSolr.BATCH_SIZE, "2");
-        runner.run(1, true, true);
+        runner.run(1, false, true);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 0);
         runner.clearTransferState();
     }
 
     @Test
-    public void testStateCleared() throws IOException, SolrServerException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testStateCleared() throws IOException {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.BATCH_SIZE, "1");
@@ -292,7 +282,7 @@ public class TestGetSolr {
 
         // run with cleared statemanager
         runner.getStateManager().clear(Scope.CLUSTER);
-        runner.run(1, true, true);
+        runner.run(1, false, true);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 10);
         runner.clearTransferState();
@@ -300,7 +290,7 @@ public class TestGetSolr {
 
     @Test
     public void testRecordWriter() throws IOException, InitializationException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.RETURN_TYPE, GetSolr.MODE_REC.getValue());
@@ -318,39 +308,31 @@ public class TestGetSolr {
         runner.enableControllerService(jsonWriter);
         runner.setProperty(SolrUtils.RECORD_WRITER, "writer");
 
-        runner.run(1,true, true);
+        runner.run(1,false, true);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 1);
         runner.assertAllFlowFilesContainAttribute(CoreAttributes.MIME_TYPE.key());
 
-        // Check for valid json
-        JsonReader reader = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(GetSolr.REL_SUCCESS).get(0)))));
-        reader.beginArray();
-        int controlScore = 0;
-        while (reader.hasNext()) {
-            reader.beginObject();
-            while (reader.hasNext()) {
-                if (reader.nextName().equals("integer_single"))
-                    controlScore += reader.nextInt();
-                else
-                    reader.skipValue();
-            }
-            reader.endObject();
-        }
-        assertEquals(controlScore, 45);
+        final MockFlowFile resultsFlowFile = runner.getFlowFilesForRelationship(GetSolr.REL_SUCCESS).get(0);
+        final ObjectMapper objectMapper = new ObjectMapper();
+        final JsonNode resultsNode = objectMapper.readTree(resultsFlowFile.getContent());
+
+        final int total = StreamSupport.stream(resultsNode.spliterator(), false)
+                .mapToInt(node -> node.get("integer_single").asInt())
+                .sum();
+        assertEquals(45, total);
     }
 
     @Test
-    public void testForValidXml() throws IOException, SolrServerException, InitializationException {
-        final org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor proc = new org.apache.nifi.processors.solr.TestGetSolr.TestableProcessor(solrClient);
+    public void testForValidXml() {
+        final TestableProcessor proc = new TestableProcessor(solrClient);
 
         TestRunner runner = createDefaultTestRunner(proc);
         runner.setProperty(GetSolr.SOLR_QUERY, "id:doc1");
         runner.setProperty(GetSolr.RETURN_FIELDS, "id");
         runner.setProperty(GetSolr.BATCH_SIZE, "10");
 
-        runner.run(1,true, true);
+        runner.run(1,false, true);
         runner.assertQueueEmpty();
         runner.assertAllFlowFilesTransferred(GetSolr.REL_SUCCESS, 1);
         runner.assertAllFlowFilesContainAttribute(CoreAttributes.MIME_TYPE.key());
@@ -359,17 +341,16 @@ public class TestGetSolr {
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(GetSolr.REL_SUCCESS).get(0)))));
     }
 
+    private static class TestableProcessor extends GetSolr {
+        private final SolrClient testSolrClient;
 
-    // Override createSolrClient and return the passed in SolrClient
-    private class TestableProcessor extends GetSolr {
-        private SolrClient solrClient;
-
-        public TestableProcessor(SolrClient solrClient) {
-            this.solrClient = solrClient;
+        public TestableProcessor(SolrClient testSolrClient) {
+            this.testSolrClient = testSolrClient;
         }
+
         @Override
         protected SolrClient createSolrClient(ProcessContext context, String solrLocation) {
-            return solrClient;
+            return testSolrClient;
         }
     }
 }
diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
index cd52c0e..9b7a7a4 100644
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/TestQuerySolr.java
@@ -19,7 +19,9 @@
 
 package org.apache.nifi.processors.solr;
 
-import com.google.gson.stream.JsonReader;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.json.JsonRecordSetWriter;
@@ -31,70 +33,63 @@ import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.common.SolrInputDocument;
-import org.junit.Assert;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.xmlunit.matchers.CompareMatcher;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.text.SimpleDateFormat;
-import java.util.Date;
+import java.time.Instant;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
-import java.util.TimeZone;
+import java.util.stream.StreamSupport;
 
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestQuerySolr {
-    static final String DEFAULT_SOLR_CORE = "testCollection";
-    static final String SOLR_CONNECT = "http://localhost:8443/solr";
+    private static final String DEFAULT_SOLR_CORE = "testCollection";
 
-    private static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.US);
-    static {
-        DATE_FORMAT.setTimeZone(TimeZone.getTimeZone("GMT"));
-    }
+    private static final String SOLR_CONNECT = "http://localhost:8443/solr";
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    private static SolrClient solrClient;
+
+    @BeforeClass
+    public static void createSolrClient() throws Exception {
+        final String relPath = TestQuerySolr.class.getResource("/").getPath();
 
-    private SolrClient solrClient;
-
-    public SolrClient createSolrClient() {
-        try {
-            // create an EmbeddedSolrServer for the processor to use
-            String relPath = getClass().getProtectionDomain().getCodeSource()
-                    .getLocation().getFile() + "../../target";
-
-            solrClient = EmbeddedSolrServerFactory.create(EmbeddedSolrServerFactory.DEFAULT_SOLR_HOME,
-                    DEFAULT_SOLR_CORE, relPath);
-
-            for (int i = 0; i < 10; i++) {
-                SolrInputDocument doc = new SolrInputDocument();
-                doc.addField("id", "doc" + i);
-                Date date = new Date();
-                doc.addField("created", DATE_FORMAT.format(date));
-                doc.addField("string_single", "single" + i + ".1");
-                doc.addField("string_multi", "multi" + i + ".1");
-                doc.addField("string_multi", "multi" + i + ".2");
-                doc.addField("integer_single", i);
-                doc.addField("integer_multi", 1);
-                doc.addField("integer_multi", 2);
-                doc.addField("integer_multi", 3);
-                doc.addField("double_single", 0.5 + i);
-
-                solrClient.add(doc);
-            }
-            solrClient.commit();
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail(e.getMessage());
+        solrClient = EmbeddedSolrServerFactory.create(EmbeddedSolrServerFactory.DEFAULT_SOLR_HOME,
+                DEFAULT_SOLR_CORE, relPath);
+
+        for (int i = 0; i < 10; i++) {
+            SolrInputDocument doc = new SolrInputDocument();
+            doc.addField("id", "doc" + i);
+            doc.addField("created", Instant.now().toString());
+            doc.addField("string_single", "single" + i + ".1");
+            doc.addField("string_multi", "multi" + i + ".1");
+            doc.addField("string_multi", "multi" + i + ".2");
+            doc.addField("integer_single", i);
+            doc.addField("integer_multi", 1);
+            doc.addField("integer_multi", 2);
+            doc.addField("integer_multi", 3);
+            doc.addField("double_single", 0.5 + i);
+
+            solrClient.add(doc);
         }
+        solrClient.commit();
+    }
 
-        return solrClient;
+    @AfterClass
+    public static void closeSolrClient() throws IOException {
+        solrClient.close();
     }
 
     private TestRunner createRunnerWithSolrClient(SolrClient solrClient) {
@@ -153,7 +148,6 @@ public class TestQuerySolr {
 
     @Test
     public void testAllFacetCategories() throws IOException {
-        SolrClient solrClient = createSolrClient();
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty("facet", "true");
@@ -170,142 +164,86 @@ public class TestQuerySolr {
         runner.setProperty("facet.query.3", "integer_multi:3");
 
         runner.enqueue(new ByteArrayInputStream(new byte[0]));
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
 
-        JsonReader reader = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.FACETS).get(0)))));
-        reader.beginObject();
-        while (reader.hasNext()) {
-            String name = reader.nextName();
-            if (name.equals("facet_queries")) {
-                assertEquals(30, returnCheckSumForArrayOfJsonObjects(reader));
-            } else if (name.equals("facet_fields")) {
-                reader.beginObject();
-                assertEquals(reader.nextName(), "integer_multi");
-                assertEquals(returnCheckSumForArrayOfJsonObjects(reader), 30);
-                reader.endObject();
-            } else if (name.equals("facet_ranges")) {
-                reader.beginObject();
-                assertEquals(reader.nextName(), "created");
-                assertEquals(returnCheckSumForArrayOfJsonObjects(reader), 10);
-                reader.endObject();
-            } else if (name.equals("facet_intervals")) {
-                reader.beginObject();
-                assertEquals(reader.nextName(), "integer_single");
-                assertEquals(returnCheckSumForArrayOfJsonObjects(reader), 7);
-                reader.endObject();
-            }
-        }
-        reader.endObject();
-        reader.close();
-        solrClient.close();
-    }
+        final MockFlowFile facetsFlowFile = runner.getFlowFilesForRelationship(QuerySolr.FACETS).get(0);
+        final JsonNode facetsNode = OBJECT_MAPPER.readTree(facetsFlowFile.getContent());
 
-    private int returnCheckSumForArrayOfJsonObjects(JsonReader reader) throws IOException {
-        int checkSum = 0;
-        reader.beginArray();
-        while (reader.hasNext()) {
-            reader.beginObject();
-            while (reader.hasNext()) {
-                if (reader.nextName().equals("count")) {
-                    checkSum += reader.nextInt();
-                } else {
-                    reader.skipValue();
-                }
-            }
-            reader.endObject();
-        }
-        reader.endArray();
-        return checkSum;
+        final int facetQueriesCount = StreamSupport.stream(facetsNode.get("facet_queries").spliterator(), false)
+                .mapToInt(node -> node.get("count").asInt())
+                .sum();
+        assertEquals(30, facetQueriesCount);
+
+        final int facetFieldsCount = StreamSupport.stream(facetsNode.get("facet_fields").get("integer_multi").spliterator(), false)
+                .mapToInt(node -> node.get("count").asInt())
+                .sum();
+        assertEquals(30, facetFieldsCount);
+
+        final int facetRangesCount = StreamSupport.stream(facetsNode.get("facet_ranges").get("created").spliterator(), false)
+                .mapToInt(node -> node.get("count").asInt())
+                .sum();
+        assertEquals(10, facetRangesCount);
+
+        final int facetIntervalsCount = StreamSupport.stream(facetsNode.get("facet_intervals").get("integer_single").spliterator(), false)
+                .mapToInt(node -> node.get("count").asInt())
+                .sum();
+        assertEquals(7, facetIntervalsCount);
     }
 
     @Test
     public void testFacetTrueButNull() throws IOException {
-        SolrClient solrClient = createSolrClient();
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty("facet", "true");
         runner.setProperty("stats", "true");
 
         runner.enqueue(new ByteArrayInputStream(new byte[0]));
-        runner.run();
+        runner.run(1, false);
 
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
         runner.assertTransferCount(QuerySolr.STATS, 1);
 
-        // Check for empty nestet Objects in JSON
-        JsonReader reader = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.FACETS).get(0)))));
-        reader.beginObject();
-        while (reader.hasNext()) {
-            if (reader.nextName().equals("facet_queries")) {
-                reader.beginArray();
-                assertFalse(reader.hasNext());
-                reader.endArray();
-            } else {
-                reader.beginObject();
-                assertFalse(reader.hasNext());
-                reader.endObject();
-            }
-        }
-        reader.endObject();
-
-        JsonReader reader_stats = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.STATS).get(0)))));
-        reader_stats.beginObject();
-        assertEquals(reader_stats.nextName(), "stats_fields");
-        reader_stats.beginObject();
-        assertFalse(reader_stats.hasNext());
-        reader_stats.endObject();
-        reader_stats.endObject();
-
-        reader.close();
-        reader_stats.close();
-        solrClient.close();
+        final MockFlowFile facetsFlowFile = runner.getFlowFilesForRelationship(QuerySolr.FACETS).get(0);
+        final JsonNode facetsNode = OBJECT_MAPPER.readTree(facetsFlowFile.getContent());
+        assertTrue("Facet Queries found", facetsNode.get("facet_queries").isEmpty());
+
+        final MockFlowFile statsFlowFile = runner.getFlowFilesForRelationship(QuerySolr.STATS).get(0);
+        final JsonNode statsNode = OBJECT_MAPPER.readTree(statsFlowFile.getContent());
+        assertTrue("Stats found", statsNode.get("stats_fields").isEmpty());
     }
 
     @Test
     public void testStats() throws IOException {
-        SolrClient solrClient = createSolrClient();
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty("stats", "true");
-        runner.setProperty("stats.field", "integer_single");
+
+        final String statsField = "integer_single";
+        runner.setProperty("stats.field",statsField);
 
         runner.enqueue(new ByteArrayInputStream(new byte[0]));
-        runner.run();
+        runner.run(1, false);
 
         runner.assertTransferCount(QuerySolr.STATS, 1);
-        JsonReader reader = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.STATS).get(0)))));
-        reader.beginObject();
-        assertEquals(reader.nextName(), "stats_fields");
-        reader.beginObject();
-        assertEquals(reader.nextName(), "integer_single");
-        reader.beginObject();
-        while (reader.hasNext()) {
-            String name = reader.nextName();
-            switch (name) {
-                case "min": assertEquals(reader.nextString(), "0.0"); break;
-                case "max": assertEquals(reader.nextString(), "9.0"); break;
-                case "count": assertEquals(reader.nextInt(), 10); break;
-                case "sum": assertEquals(reader.nextString(), "45.0"); break;
-                default: reader.skipValue(); break;
-            }
-        }
-        reader.endObject();
-        reader.endObject();
-        reader.endObject();
 
-        reader.close();
-        solrClient.close();
+        final MockFlowFile statsFlowFile = runner.getFlowFilesForRelationship(QuerySolr.STATS).get(0);
+        final JsonNode statsNode = OBJECT_MAPPER.readTree(statsFlowFile.getContent());
+
+        final JsonNode statsFieldsNode = statsNode.get("stats_fields");
+        assertNotNull("Stats Fields not found", statsFieldsNode);
+        final JsonNode configuredStatsFieldNode = statsFieldsNode.get(statsField);
+
+        assertEquals("0.0", configuredStatsFieldNode.get("min").asText());
+        assertEquals("9.0", configuredStatsFieldNode.get("max").asText());
+        assertEquals("10", configuredStatsFieldNode.get("count").asText());
+        assertEquals("45.0", configuredStatsFieldNode.get("sum").asText());
+        assertEquals("4.5", configuredStatsFieldNode.get("mean").asText());
     }
 
     @Test
-    public void testRelationshipRoutings() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testRelationshipRoutings() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty("facet", "true");
@@ -355,7 +293,7 @@ public class TestQuerySolr {
         // Processor has an input connection and succeeds
         runner.setNonLoopConnection(true);
         runner.enqueue(new byte[0]);
-        runner.run(1, true);
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
         runner.assertTransferCount(QuerySolr.STATS, 1);
@@ -378,13 +316,10 @@ public class TestQuerySolr {
         flowFile.assertAttributeExists(QuerySolr.ATTRIBUTE_CURSOR_MARK);
         flowFile.assertAttributeExists(QuerySolr.ATTRIBUTE_QUERY_TIME);
         runner.clearTransferState();
-
-        solrClient.close();
     }
 
     @Test
-    public void testExpressionLanguageForProperties() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testExpressionLanguageForProperties() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(SolrUtils.SOLR_TYPE, SolrUtils.SOLR_TYPE_CLOUD.getValue());
@@ -395,7 +330,7 @@ public class TestQuerySolr {
         runner.setProperty(QuerySolr.SOLR_PARAM_START, "${start}");
         runner.setProperty(QuerySolr.SOLR_PARAM_ROWS, "${rows}");
 
-        runner.enqueue(new byte[0], new HashMap<String,String>(){{
+        runner.enqueue(new byte[0], new HashMap<String, String>(){{
             put("query", "id:(doc0 OR doc1 OR doc2 OR doc3)");
             put("handler", "/select");
             put("fields", "id");
@@ -403,18 +338,15 @@ public class TestQuerySolr {
             put("start", "1");
             put("rows", "2");
         }});
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
 
         String expectedXml = "<docs><doc><field name=\"id\">doc2</field></doc><doc><field name=\"id\">doc1</field></doc></docs>";
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0)))));
-
-        solrClient.close();
     }
 
     @Test
-    public void testSingleFilterQuery() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testSingleFilterQuery() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
         runner.setProperty(QuerySolr.SOLR_PARAM_SORT, "id asc");
         runner.setProperty(QuerySolr.SOLR_PARAM_FIELD_LIST, "id");
@@ -422,19 +354,16 @@ public class TestQuerySolr {
         runner.setProperty("fq", "id:(doc2 OR doc3)");
 
         runner.enqueue(new byte[0]);
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
 
         String expectedXml = "<docs><doc><field name=\"id\">doc2</field></doc><doc><field name=\"id\">doc3</field></doc></docs>";
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0)))));
-
-        solrClient.close();
     }
 
 
     @Test
-    public void testMultipleFilterQueries() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testMultipleFilterQueries() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
         runner.setProperty(QuerySolr.SOLR_PARAM_SORT, "id asc");
         runner.setProperty(QuerySolr.SOLR_PARAM_FIELD_LIST, "id");
@@ -444,18 +373,15 @@ public class TestQuerySolr {
         runner.setProperty("fq.3", "id:(doc2 OR doc3 OR doc4 OR doc5)");
 
         runner.enqueue(new byte[0]);
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
 
         String expectedXml = "<docs><doc><field name=\"id\">doc2</field></doc><doc><field name=\"id\">doc3</field></doc></docs>";
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0)))));
-
-        solrClient.close();
     }
 
     @Test
-    public void testStandardResponse() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testStandardResponse() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_QUERY, "id:(doc0 OR doc1)");
@@ -463,7 +389,7 @@ public class TestQuerySolr {
         runner.setProperty(QuerySolr.SOLR_PARAM_SORT, "id desc");
 
         runner.setNonLoopConnection(false);
-        runner.run();
+        runner.run(1, false);
         runner.assertAllFlowFilesTransferred(QuerySolr.RESULTS, 1);
 
         MockFlowFile flowFile = runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0);
@@ -473,13 +399,10 @@ public class TestQuerySolr {
 
         String expectedXml = "<docs><doc><field name=\"id\">doc1</field></doc><doc><field name=\"id\">doc0</field></doc></docs>";
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(flowFile))));
-
-        solrClient.close();
     }
 
     @Test
-    public void testPreserveOriginalContent() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testPreserveOriginalContent() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_QUERY, "id:doc0");
@@ -488,20 +411,17 @@ public class TestQuerySolr {
         String content = "test content 123";
 
         runner.enqueue(content);
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
         runner.assertTransferCount(QuerySolr.ORIGINAL, 1);
 
         String expectedXml = "<docs><doc><field name=\"id\">doc0</field></doc></docs>";
         assertThat(expectedXml, CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0)))));
         assertEquals(content, new String(runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.ORIGINAL).get(0))));
-
-        solrClient.close();
     }
 
     @Test
-    public void testRetrievalOfFullResults() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testRetrievalOfFullResults() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_FIELD_LIST, "id");
@@ -510,22 +430,22 @@ public class TestQuerySolr {
         runner.setProperty(QuerySolr.AMOUNT_DOCUMENTS_TO_RETURN, QuerySolr.RETURN_ALL_RESULTS);
 
         runner.enqueue(new byte[0]);
-        runner.run();
+        runner.run(1, false);
         runner.assertTransferCount(QuerySolr.RESULTS, 5);
         runner.assertTransferCount(QuerySolr.ORIGINAL, 1);
         runner.assertTransferCount(QuerySolr.STATS, 0);
         runner.assertTransferCount(QuerySolr.FACETS, 0);
 
         List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(QuerySolr.RESULTS);
-        Integer documentCounter = 0;
-        Integer startParam = 0;
+        int documentCounter = 0;
+        int startParam = 0;
 
         for (MockFlowFile flowFile : flowFiles) {
             Map<String,String> attributes = flowFile.getAttributes();
-            assertEquals(attributes.get(QuerySolr.ATTRIBUTE_SOLR_START), startParam.toString());
+            assertEquals(attributes.get(QuerySolr.ATTRIBUTE_SOLR_START), Integer.toString(startParam));
             startParam += 2;
 
-            StringBuffer expectedXml = new StringBuffer()
+            StringBuilder expectedXml = new StringBuilder()
                     .append("<docs><doc><field name=\"id\">doc")
                     .append(documentCounter++)
                     .append("</field></doc><doc><field name=\"id\">doc")
@@ -533,13 +453,10 @@ public class TestQuerySolr {
                     .append("</field></doc></docs>");
             assertThat(expectedXml.toString(), CompareMatcher.isIdenticalTo(new String(runner.getContentAsByteArray(flowFile))));
         }
-
-        solrClient.close();
     }
 
     @Test
-    public void testRetrievalOfFullResults2() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testRetrievalOfFullResults2() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_FIELD_LIST, "id");
@@ -550,19 +467,16 @@ public class TestQuerySolr {
         runner.setProperty("stats", "true");
 
         runner.enqueue(new byte[0]);
-        runner.run();
+        runner.run(1, false);
 
         runner.assertTransferCount(QuerySolr.RESULTS, 4);
         runner.assertTransferCount(QuerySolr.ORIGINAL, 1);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
         runner.assertTransferCount(QuerySolr.STATS, 1);
-
-        solrClient.close();
     }
 
     @Test
-    public void testRetrievalOfFullResults3() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testRetrievalOfFullResults3() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_FIELD_LIST, "id");
@@ -573,20 +487,17 @@ public class TestQuerySolr {
         runner.setProperty("stats", "true");
 
         runner.setNonLoopConnection(false);
-        runner.run();
+        runner.run(1, false);
 
         runner.assertTransferCount(QuerySolr.RESULTS, 4);
         runner.assertTransferCount(QuerySolr.ORIGINAL, 0);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
         runner.assertTransferCount(QuerySolr.STATS, 1);
-
-        solrClient.close();
     }
 
 
     @Test
     public void testRecordResponse() throws IOException, InitializationException {
-        SolrClient solrClient = createSolrClient();
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.RETURN_TYPE, QuerySolr.MODE_REC.getValue());
@@ -606,54 +517,38 @@ public class TestQuerySolr {
 
         runner.setNonLoopConnection(false);
 
-        runner.run(1);
+        runner.run(1, false);
         runner.assertQueueEmpty();
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
 
-        JsonReader reader = new JsonReader(new InputStreamReader(new ByteArrayInputStream(
-                runner.getContentAsByteArray(runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0)))));
-        reader.beginArray();
-        int controlScore = 0;
-        while (reader.hasNext()) {
-            reader.beginObject();
-            while (reader.hasNext()) {
-                if (reader.nextName().equals("integer_single")) {
-                    controlScore += reader.nextInt();
-                } else {
-                    reader.skipValue();
-                }
-            }
-            reader.endObject();
-        }
-        reader.close();
-        solrClient.close();
+        final MockFlowFile resultsFlowFile = runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0);
+        final JsonNode resultsNode = OBJECT_MAPPER.readTree(resultsFlowFile.getContent());
 
-        assertEquals(controlScore, 45);
+        final int total = StreamSupport.stream(resultsNode.spliterator(), false)
+                .mapToInt(node -> node.get("integer_single").asInt())
+                .sum();
+        assertEquals(45, total);
     }
 
     @Test
-    public void testExceedStartParam() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testExceedStartParam() {
         TestRunner runner = createRunnerWithSolrClient(solrClient);
 
         runner.setProperty(QuerySolr.SOLR_PARAM_START, "10001");
 
         runner.setNonLoopConnection(false);
 
-        runner.run();
+        runner.run(1, false);
         runner.assertAllFlowFilesTransferred(QuerySolr.RESULTS, 1);
 
         MockFlowFile flowFile = runner.getFlowFilesForRelationship(QuerySolr.RESULTS).get(0);
 
         assertEquals("10001", flowFile.getAttribute(QuerySolr.ATTRIBUTE_SOLR_START));
         assertEquals(0, runner.getContentAsByteArray(flowFile).length);
-
-        solrClient.close();
     }
 
     @Test
-    public void testAttributesFailure() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testAttributesFailure() {
         TestRunner runner = createRunnerWithSolrCloudClient(solrClient);
 
         runner.setProperty("facet", "true");
@@ -661,7 +556,7 @@ public class TestQuerySolr {
         runner.setProperty(QuerySolr.SOLR_PARAM_REQUEST_HANDLER, "/nonexistentrequesthandler");
 
         runner.enqueue("");
-        runner.run();
+        runner.run(1, false);
 
         runner.assertAllFlowFilesTransferred(QuerySolr.FAILURE, 1);
 
@@ -675,15 +570,14 @@ public class TestQuerySolr {
     }
 
     @Test
-    public void testAttributes() throws IOException {
-        SolrClient solrClient = createSolrClient();
+    public void testAttributes() {
         TestRunner runner = createRunnerWithSolrCloudClient(solrClient);
 
         runner.setProperty("facet", "true");
         runner.setProperty("stats", "true");
 
         runner.enqueue("");
-        runner.run();
+        runner.run(1, false);
 
         runner.assertTransferCount(QuerySolr.RESULTS, 1);
         runner.assertTransferCount(QuerySolr.FACETS, 1);
@@ -771,20 +665,18 @@ public class TestQuerySolr {
 
         assertEquals(SOLR_CONNECT, attributes.get(QuerySolr.ATTRIBUTE_SOLR_CONNECT));
         assertEquals(DEFAULT_SOLR_CORE, attributes.get(QuerySolr.ATTRIBUTE_SOLR_COLLECTION));
-
-        solrClient.close();
     }
 
-    // Override createSolrClient and return the passed in SolrClient
-    private class TestableProcessor extends QuerySolr {
-        private SolrClient solrClient;
+    private static class TestableProcessor extends QuerySolr {
+        private final SolrClient testSolrClient;
 
-        public TestableProcessor(SolrClient solrClient) {
-            this.solrClient = solrClient;
+        public TestableProcessor(SolrClient testSolrClient) {
+            this.testSolrClient = testSolrClient;
         }
+
         @Override
         protected SolrClient createSolrClient(ProcessContext context, String solrLocation) {
-            return solrClient;
+            return testSolrClient;
         }
     }
 }