You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2018/02/26 06:13:53 UTC

[2/3] lucene-solr:master: SOLR-10809: Get precommit lint warnings out of Solr core

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
index 20c2f1a..8aeaa73 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
@@ -130,196 +130,200 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
   @ShardsFixed(num = 4)
   public void test() throws Exception {
     del("*:*");
-    
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    
-    ModifiableSolrParams params = new ModifiableSolrParams();
 
+    ModifiableSolrParams params = new ModifiableSolrParams();
     List<Node> nodes = new ArrayList<>();
+    AddUpdateCommand cmd = new AddUpdateCommand(null);
+    List<Error> errors;
+    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
+    long numFound;
+    HttpSolrClient client;
+    ZkNodeProps nodeProps;
 
-    ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
-        ((HttpSolrClient) controlClient).getBaseURL(),
-        ZkStateReader.CORE_NAME_PROP, "");
-    nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
 
-    // add one doc to controlClient
-    
-    AddUpdateCommand cmd = new AddUpdateCommand(null);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    params = new ModifiableSolrParams();
+      nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+          ((HttpSolrClient) controlClient).getBaseURL(),
+          ZkStateReader.CORE_NAME_PROP, "");
+      nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
 
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    
-    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
+      // add one doc to controlClient
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      params = new ModifiableSolrParams();
 
-    
-    List<Error> errors = cmdDistrib.getErrors();
-    
-    assertEquals(errors.toString(), 0, errors.size());
-    
-    long numFound = controlClient.query(new SolrQuery("*:*")).getResults()
-        .getNumFound();
-    assertEquals(1, numFound);
-    
-    HttpSolrClient client = (HttpSolrClient) clients.get(0);
-    nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
-        client.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-    nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
-    
-    // add another 2 docs to control and 3 to client
-    cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    
-    int id2 = id.incrementAndGet();
-    AddUpdateCommand cmd2 = new AddUpdateCommand(null);
-    cmd2.solrDoc = sdoc("id", id2);
+      cmdDistrib.distribAdd(cmd, nodes, params);
 
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribAdd(cmd2, nodes, params);
-    
-    AddUpdateCommand cmd3 = new AddUpdateCommand(null);
-    cmd3.solrDoc = sdoc("id", id.incrementAndGet());
-    
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribAdd(cmd3, Collections.singletonList(nodes.get(1)), params);
-    
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
-    errors = cmdDistrib.getErrors();
-    
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+
+
+      errors = cmdDistrib.getErrors();
+
+      assertEquals(errors.toString(), 0, errors.size());
+
+      numFound = controlClient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound();
+      assertEquals(1, numFound);
+
+      client = (HttpSolrClient) clients.get(0);
+      nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+          client.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+      nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
+    }
+    int id2;
+    // add another 2 docs to control and 3 to client
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribAdd(cmd, nodes, params);
+
+      id2 = id.incrementAndGet();
+      AddUpdateCommand cmd2 = new AddUpdateCommand(null);
+      cmd2.solrDoc = sdoc("id", id2);
+
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribAdd(cmd2, nodes, params);
+
+      AddUpdateCommand cmd3 = new AddUpdateCommand(null);
+      cmd3.solrDoc = sdoc("id", id.incrementAndGet());
+
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribAdd(cmd3, Collections.singletonList(nodes.get(1)), params);
+
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+      errors = cmdDistrib.getErrors();
+    }
     assertEquals(errors.toString(), 0, errors.size());
-    
+
     SolrDocumentList results = controlClient.query(new SolrQuery("*:*")).getResults();
     numFound = results.getNumFound();
     assertEquals(results.toString(), 3, numFound);
-    
+
     numFound = client.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
     assertEquals(3, numFound);
-    
+
     // now delete doc 2 which is on both control and client1
-    
+
     DeleteUpdateCommand dcmd = new DeleteUpdateCommand(null);
     dcmd.id = Integer.toString(id2);
-    
 
-    cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    
-    cmdDistrib.distribDelete(dcmd, nodes, params);
-    
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
 
-    errors = cmdDistrib.getErrors();
-    
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
+
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+
+      cmdDistrib.distribDelete(dcmd, nodes, params);
+
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+
+      errors = cmdDistrib.getErrors();
+    }
+
     assertEquals(errors.toString(), 0, errors.size());
-    
-    
+
+
     results = controlClient.query(new SolrQuery("*:*")).getResults();
     numFound = results.getNumFound();
     assertEquals(results.toString(), 2, numFound);
-    
+
     numFound = client.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
     assertEquals(results.toString(), 2, numFound);
-    
+
     for (SolrClient c : clients) {
       c.optimize();
       //System.out.println(clients.get(0).request(new LukeRequest()));
     }
-    
-    cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    
-    int cnt = atLeast(303);
-    for (int i = 0; i < cnt; i++) {
+
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
+
+      int cnt = atLeast(303);
+      for (int i = 0; i < cnt; i++) {
+        nodes.clear();
+        for (SolrClient c : clients) {
+          if (random().nextBoolean()) {
+            continue;
+          }
+          HttpSolrClient httpClient = (HttpSolrClient) c;
+          nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+              httpClient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+          nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
+
+        }
+        AddUpdateCommand c = new AddUpdateCommand(null);
+        c.solrDoc = sdoc("id", id.incrementAndGet());
+        if (nodes.size() > 0) {
+          params = new ModifiableSolrParams();
+          cmdDistrib.distribAdd(c, nodes, params);
+        }
+      }
+
       nodes.clear();
+
       for (SolrClient c : clients) {
-        if (random().nextBoolean()) {
-          continue;
-        }
         HttpSolrClient httpClient = (HttpSolrClient) c;
         nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
             httpClient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-        nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
 
+        nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
       }
-      AddUpdateCommand c = new AddUpdateCommand(null);
-      c.solrDoc = sdoc("id", id.incrementAndGet());
-      if (nodes.size() > 0) {
-        params = new ModifiableSolrParams();
-        cmdDistrib.distribAdd(c, nodes, params);
-      }
-    }
-    
-    nodes.clear();
-    
-    for (SolrClient c : clients) {
-      HttpSolrClient httpClient = (HttpSolrClient) c;
-      nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
-          httpClient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-      
-      nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
-    }
-    
-    final AtomicInteger commits = new AtomicInteger();
-    for(JettySolrRunner jetty : jettys) {
-      CoreContainer cores = jetty.getCoreContainer();
-      try (SolrCore core = cores.getCore("collection1")) {
-        core.getUpdateHandler().registerCommitCallback(new SolrEventListener() {
-          @Override
-          public void init(NamedList args) {
-          }
 
-          @Override
-          public void postSoftCommit() {
-          }
+      final AtomicInteger commits = new AtomicInteger();
+      for (JettySolrRunner jetty : jettys) {
+        CoreContainer cores = jetty.getCoreContainer();
+        try (SolrCore core = cores.getCore("collection1")) {
+          core.getUpdateHandler().registerCommitCallback(new SolrEventListener() {
+            @Override
+            public void init(NamedList args) {
+            }
+
+            @Override
+            public void postSoftCommit() {
+            }
+
+            @Override
+            public void postCommit() {
+              commits.incrementAndGet();
+            }
+
+            @Override
+            public void newSearcher(SolrIndexSearcher newSearcher,
+                                    SolrIndexSearcher currentSearcher) {
+            }
+          });
+        }
+      }
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
 
-          @Override
-          public void postCommit() {
-            commits.incrementAndGet();
-          }
+      cmdDistrib.distribCommit(ccmd, nodes, params);
 
-          @Override
-          public void newSearcher(SolrIndexSearcher newSearcher,
-                                  SolrIndexSearcher currentSearcher) {
-          }
-        });
-      }
-    }
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.finish();
 
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    
-    cmdDistrib.finish();
+      assertEquals(getShardCount(), commits.get());
 
-    assertEquals(getShardCount(), commits.get());
-    
-    for (SolrClient c : clients) {
-      NamedList<Object> resp = c.request(new LukeRequest());
-      assertEquals("SOLR-3428: We only did adds - there should be no deletes",
-          ((NamedList<Object>) resp.get("index")).get("numDocs"),
-          ((NamedList<Object>) resp.get("index")).get("maxDoc"));
+      for (SolrClient c : clients) {
+        NamedList<Object> resp = c.request(new LukeRequest());
+        assertEquals("SOLR-3428: We only did adds - there should be no deletes",
+            ((NamedList<Object>) resp.get("index")).get("numDocs"),
+            ((NamedList<Object>) resp.get("index")).get("maxDoc"));
+      }
     }
     
-    
     testMaxRetries();
     testOneRetry();
     testRetryNodeAgainstBadAddress();
@@ -330,33 +334,34 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
 
   private void testMaxRetries() throws IOException {
     final MockStreamingSolrClients streamingClients = new MockStreamingSolrClients(updateShardHandler);
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0);
-    streamingClients.setExp(Exp.CONNECT_EXCEPTION);
-    ArrayList<Node> nodes = new ArrayList<>();
-    final HttpSolrClient solrclient1 = (HttpSolrClient) clients.get(0);
-    
-    final AtomicInteger retries = new AtomicInteger();
-    ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient1.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-    RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
-      @Override
-      public boolean checkRetry() {
-        retries.incrementAndGet();
-        return true;
-      }
-    };
-    
-    nodes.add(retryNode);
-    
-    AddUpdateCommand cmd = new AddUpdateCommand(null);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    ModifiableSolrParams params = new ModifiableSolrParams();
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0)) {
+      streamingClients.setExp(Exp.CONNECT_EXCEPTION);
+      ArrayList<Node> nodes = new ArrayList<>();
+      final HttpSolrClient solrclient1 = (HttpSolrClient) clients.get(0);
+
+      final AtomicInteger retries = new AtomicInteger();
+      ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient1.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+      RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
+        @Override
+        public boolean checkRetry() {
+          retries.incrementAndGet();
+          return true;
+        }
+      };
 
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    cmdDistrib.finish();
-    
-    assertEquals(6, retries.get());
-    
-    assertEquals(1, cmdDistrib.getErrors().size());
+      nodes.add(retryNode);
+
+      AddUpdateCommand cmd = new AddUpdateCommand(null);
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      cmdDistrib.distribAdd(cmd, nodes, params);
+      cmdDistrib.finish();
+
+      assertEquals(6, retries.get());
+
+      assertEquals(1, cmdDistrib.getErrors().size());
+    }
   }
   
   private void testOneRetry() throws Exception {
@@ -364,45 +369,46 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
     long numFoundBefore = solrclient.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
     final MockStreamingSolrClients streamingClients = new MockStreamingSolrClients(updateShardHandler);
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0);
-    streamingClients.setExp(Exp.CONNECT_EXCEPTION);
-    ArrayList<Node> nodes = new ArrayList<>();
-
-    ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
-        ZkStateReader.CORE_NAME_PROP, "");
-
-    final AtomicInteger retries = new AtomicInteger();
-    nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-    RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
-      @Override
-      public boolean checkRetry() {
-        streamingClients.setExp(null);
-        retries.incrementAndGet();
-        return true;
-      }
-    };
-    
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0)) {
+      streamingClients.setExp(Exp.CONNECT_EXCEPTION);
+      ArrayList<Node> nodes = new ArrayList<>();
+
+      ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
+          ZkStateReader.CORE_NAME_PROP, "");
+
+      final AtomicInteger retries = new AtomicInteger();
+      nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+      RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
+        @Override
+        public boolean checkRetry() {
+          streamingClients.setExp(null);
+          retries.incrementAndGet();
+          return true;
+        }
+      };
 
-    nodes.add(retryNode);
-    
-    AddUpdateCommand cmd = new AddUpdateCommand(null);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    ModifiableSolrParams params = new ModifiableSolrParams();
 
-    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
-    
-    assertEquals(1, retries.get());
-    
-    
-    long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
-        .getNumFound();
-    
-    // we will get java.net.ConnectException which we retry on
-    assertEquals(numFoundBefore + 1, numFoundAfter);
-    assertEquals(0, cmdDistrib.getErrors().size());
+      nodes.add(retryNode);
+
+      AddUpdateCommand cmd = new AddUpdateCommand(null);
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
+      cmdDistrib.distribAdd(cmd, nodes, params);
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+
+      assertEquals(1, retries.get());
+
+
+      long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound();
+
+      // we will get java.net.ConnectException which we retry on
+      assertEquals(numFoundBefore + 1, numFoundAfter);
+      assertEquals(0, cmdDistrib.getErrors().size());
+    }
   }
 
   private void testRetryNodeWontRetrySocketError() throws Exception {
@@ -410,100 +416,101 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
     long numFoundBefore = solrclient.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
     final MockStreamingSolrClients streamingClients = new MockStreamingSolrClients(updateShardHandler);
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0);
-    streamingClients.setExp(Exp.SOCKET_EXCEPTION);
-    ArrayList<Node> nodes = new ArrayList<>();
-
-    ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
-        ZkStateReader.CORE_NAME_PROP, "");
-
-    final AtomicInteger retries = new AtomicInteger();
-    nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
-    RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
-      @Override
-      public boolean checkRetry() {
-        retries.incrementAndGet();
-        return true;
-      }
-    };
-    
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(streamingClients, 5, 0)) {
+      streamingClients.setExp(Exp.SOCKET_EXCEPTION);
+      ArrayList<Node> nodes = new ArrayList<>();
+
+      ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
+          ZkStateReader.CORE_NAME_PROP, "");
+
+      final AtomicInteger retries = new AtomicInteger();
+      nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+      RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
+        @Override
+        public boolean checkRetry() {
+          retries.incrementAndGet();
+          return true;
+        }
+      };
 
-    nodes.add(retryNode);
-    
-    AddUpdateCommand cmd = new AddUpdateCommand(null);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    ModifiableSolrParams params = new ModifiableSolrParams();
 
-    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    
-    streamingClients.setExp(null);
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
-    
-    // it will checkRetry, but not actually do it...
-    assertEquals(1, retries.get());
-    
-    
-    long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
-        .getNumFound();
-    
-    // we will get java.net.SocketException: Network is unreachable, which we don't retry on
-    assertEquals(numFoundBefore, numFoundAfter);
-    assertEquals(1, cmdDistrib.getErrors().size());
+      nodes.add(retryNode);
+
+      AddUpdateCommand cmd = new AddUpdateCommand(null);
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
+      cmdDistrib.distribAdd(cmd, nodes, params);
+
+      streamingClients.setExp(null);
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+
+      // it will checkRetry, but not actually do it...
+      assertEquals(1, retries.get());
+
+
+      long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound();
+
+      // we will get java.net.SocketException: Network is unreachable, which we don't retry on
+      assertEquals(numFoundBefore, numFoundAfter);
+      assertEquals(1, cmdDistrib.getErrors().size());
+    }
   }
 
   private void testRetryNodeAgainstBadAddress() throws SolrServerException, IOException {
     // Test RetryNode
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    final HttpSolrClient solrclient = (HttpSolrClient) clients.get(0);
-    long numFoundBefore = solrclient.query(new SolrQuery("*:*")).getResults()
-        .getNumFound();
-    
-    ArrayList<Node> nodes = new ArrayList<>();
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
+      final HttpSolrClient solrclient = (HttpSolrClient) clients.get(0);
+      long numFoundBefore = solrclient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound();
+
+      ArrayList<Node> nodes = new ArrayList<>();
+
+      ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, "[ff01::114]:33332" + context, ZkStateReader.CORE_NAME_PROP, "");
+      RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
+        @Override
+        public boolean checkRetry() {
+          ZkNodeProps leaderProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
+              ZkStateReader.CORE_NAME_PROP, "");
+          this.nodeProps = new ZkCoreNodeProps(leaderProps);
+
+          return true;
+        }
+      };
 
-    ZkNodeProps nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, "[ff01::114]:33332" + context, ZkStateReader.CORE_NAME_PROP, "");
-    RetryNode retryNode = new RetryNode(new ZkCoreNodeProps(nodeProps), null, "collection1", "shard1") {
-      @Override
-      public boolean checkRetry() {
-        ZkNodeProps leaderProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP, solrclient.getBaseURL(),
-            ZkStateReader.CORE_NAME_PROP, "");
-        this.nodeProps = new ZkCoreNodeProps(leaderProps);
 
-        return true;
-      }
-    };
-    
+      nodes.add(retryNode);
 
-    nodes.add(retryNode);
-    
-    
-    AddUpdateCommand cmd = new AddUpdateCommand(null);
-    cmd.solrDoc = sdoc("id", id.incrementAndGet());
-    ModifiableSolrParams params = new ModifiableSolrParams();
 
-    cmdDistrib.distribAdd(cmd, nodes, params);
-    
-    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
-    params = new ModifiableSolrParams();
-    params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
-    cmdDistrib.distribCommit(ccmd, nodes, params);
-    cmdDistrib.finish();
-    
-    long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
-        .getNumFound();
-    
-    // different OS's will throw different exceptions for the bad address above
-    if (numFoundBefore != numFoundAfter) {
-      assertEquals(0, cmdDistrib.getErrors().size());
-      assertEquals(numFoundBefore + 1, numFoundAfter);
-    } else {
-      // we will get java.net.SocketException: Network is unreachable and not retry
-      assertEquals(numFoundBefore, numFoundAfter);
-      
-      assertEquals(1, cmdDistrib.getErrors().size());
+      AddUpdateCommand cmd = new AddUpdateCommand(null);
+      cmd.solrDoc = sdoc("id", id.incrementAndGet());
+      ModifiableSolrParams params = new ModifiableSolrParams();
+
+      cmdDistrib.distribAdd(cmd, nodes, params);
+
+      CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
+      params = new ModifiableSolrParams();
+      params.set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
+      cmdDistrib.distribCommit(ccmd, nodes, params);
+      cmdDistrib.finish();
+
+      long numFoundAfter = solrclient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound();
+
+      // different OS's will throw different exceptions for the bad address above
+      if (numFoundBefore != numFoundAfter) {
+        assertEquals(0, cmdDistrib.getErrors().size());
+        assertEquals(numFoundBefore + 1, numFoundAfter);
+      } else {
+        // we will get java.net.SocketException: Network is unreachable and not retry
+        assertEquals(numFoundBefore, numFoundAfter);
+
+        assertEquals(1, cmdDistrib.getErrors().size());
+      }
     }
-    
   }
   
   @Override
@@ -513,21 +520,22 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
   }
 
   private void testDistribOpenSearcher() {
-    SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler);
-    UpdateRequest updateRequest = new UpdateRequest();
+    try (SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(updateShardHandler)) {
+      UpdateRequest updateRequest = new UpdateRequest();
 
-    CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
+      CommitUpdateCommand ccmd = new CommitUpdateCommand(null, false);
 
-    //test default value (should be true)
-    cmdDistrib.addCommit(updateRequest, ccmd);
-    boolean openSearcher = updateRequest.getParams().getBool(UpdateParams.OPEN_SEARCHER,false);
-    assertTrue(openSearcher);
+      //test default value (should be true)
+      cmdDistrib.addCommit(updateRequest, ccmd);
+      boolean openSearcher = updateRequest.getParams().getBool(UpdateParams.OPEN_SEARCHER, false);
+      assertTrue(openSearcher);
 
-    //test openSearcher = false
-    ccmd.openSearcher = false;
-    
-    cmdDistrib.addCommit(updateRequest, ccmd);
-    openSearcher = updateRequest.getParams().getBool(UpdateParams.OPEN_SEARCHER,true);
-    assertFalse(openSearcher);
+      //test openSearcher = false
+      ccmd.openSearcher = false;
+
+      cmdDistrib.addCommit(updateRequest, ccmd);
+      openSearcher = updateRequest.getParams().getBool(UpdateParams.OPEN_SEARCHER, true);
+      assertFalse(openSearcher);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java b/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
index 0bd2e6e..ae743da 100644
--- a/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
+++ b/solr/core/src/test/org/apache/solr/update/SolrIndexSplitterTest.java
@@ -178,8 +178,10 @@ public class SolrIndexSplitterTest extends SolrTestCaseJ4 {
       } finally {
         if (request != null) request.close();
       }
-      EmbeddedSolrServer server1 = new EmbeddedSolrServer(h.getCoreContainer(), "split1");
-      EmbeddedSolrServer server2 = new EmbeddedSolrServer(h.getCoreContainer(), "split2");
+      @SuppressWarnings("resource")
+      final EmbeddedSolrServer server1 = new EmbeddedSolrServer(h.getCoreContainer(), "split1");
+      @SuppressWarnings("resource")
+      final EmbeddedSolrServer server2 = new EmbeddedSolrServer(h.getCoreContainer(), "split2");
       server1.commit(true, true);
       server2.commit(true, true);
       assertEquals("id:dorothy should be present in split index1", 1, server1.query(new SolrQuery("id:dorothy")).getResults().getNumFound());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java b/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
index a13fdaa..1a6f60d 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/RecordingUpdateProcessorFactory.java
@@ -61,6 +61,7 @@ public final class RecordingUpdateProcessorFactory
   }
 
   @Override
+  @SuppressWarnings("resource")
   public synchronized UpdateRequestProcessor getInstance(SolrQueryRequest req, 
                                                          SolrQueryResponse rsp, 
                                                          UpdateRequestProcessor next ) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java
index 0bd4456..a4f528b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java
@@ -223,7 +223,9 @@ public class DaemonStream extends TupleStream implements Expressible {
     if(closed) {
       return;
     }
-    streamRunner.setShutdown(true);
+    if (streamRunner != null) {
+      streamRunner.setShutdown(true);
+    }
     this.closed = true;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
index 0f98e35..29e485b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
@@ -348,7 +348,9 @@ public class FacetStream extends TupleStream implements Expressible  {
 
   public void close() throws IOException {
     if(cache == null) {
-      cloudSolrClient.close();
+      if (cloudSolrClient != null) {
+        cloudSolrClient.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
index 75d2d34..b6ad276 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
@@ -305,11 +305,13 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
   }
 
   public void close() throws IOException {
-    if (isCloseCache) {
+    if (isCloseCache && cache != null) {
       cache.close();
     }
 
-    executorService.shutdown();
+    if (executorService != null) {
+      executorService.shutdown();
+    }
   }
 
   /** Return the stream sort - ie, the order in which records are returned */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
index 879c708..f56431c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
@@ -391,11 +391,13 @@ public class TextLogitStream extends TupleStream implements Expressible {
   }
 
   public void close() throws IOException {
-    if (isCloseCache) {
+    if (isCloseCache && cache != null) {
       cache.close();
     }
 
-    executorService.shutdown();
+    if (executorService != null) {
+      executorService.shutdown();
+    }
     termsStream.close();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index b7a1edd..4bb6223 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -355,7 +355,7 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
         }
       }
 
-      if (streamContext.getSolrClientCache() == null) {
+      if (streamContext != null && streamContext.getSolrClientCache() == null) {
         cloudSolrClient.close();
       }
     }
@@ -455,6 +455,9 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
 
   private void persistCheckpoints() throws IOException{
 
+    if (cloudSolrClient == null) {
+      return;
+    }
     UpdateRequest request = new UpdateRequest();
     request.setParam("collection", checkpointCollection);
     SolrInputDocument doc = new SolrInputDocument();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 5c12645..5f3d498 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -231,7 +231,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       solrParams.add("qt", "/stream");
       solrParams.add("expr", "search(myCollection, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")");
       solrParams.add("myCollection.shards", buf.toString());
-      SolrStream solrStream = new SolrStream(shardUrls.get(0), solrParams);
       stream.setStreamContext(context);
       tuples = getTuples(stream);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
index 36b2829..9327ee9 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java
@@ -70,161 +70,152 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
   @Test
   public void testCloudSolrStream() throws Exception {
 
-    CloudSolrStream stream;
     String expressionString;
     
     // Basic test
-    stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", fq=\"a_s:one\", fq=\"a_s:two\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    System.out.println("ExpressionString: " + expressionString.toString());
-    assertTrue(expressionString.contains("search(collection1,"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
-    assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
-    assertTrue(expressionString.contains("fq=\"a_s:one\""));
-    assertTrue(expressionString.contains("fq=\"a_s:two\""));
-    
+    try (CloudSolrStream stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", fq=\"a_s:one\", fq=\"a_s:two\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      System.out.println("ExpressionString: " + expressionString.toString());
+      assertTrue(expressionString.contains("search(collection1,"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
+      assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
+      assertTrue(expressionString.contains("fq=\"a_s:one\""));
+      assertTrue(expressionString.contains("fq=\"a_s:two\""));
+    }
     // Basic w/aliases
-    stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"id=izzy,a_s=kayden\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("id=izzy"));
-    assertTrue(expressionString.contains("a_s=kayden"));
-
+    try (CloudSolrStream stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"id=izzy,a_s=kayden\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("id=izzy"));
+      assertTrue(expressionString.contains("a_s=kayden"));
+    }
   }
   
   @Test
   public void testSelectStream() throws Exception {
-
-    SelectStream stream;
     String expressionString;
     
     // Basic test
-    stream = new SelectStream(StreamExpressionParser.parse("select(\"a_s as fieldA\", search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"))"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("select(search(collection1,"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
-    assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
-    assertTrue(expressionString.contains("a_s as fieldA"));
-    
+    try (SelectStream stream = new SelectStream(StreamExpressionParser.parse("select(\"a_s as fieldA\", search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"))"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("select(search(collection1,"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
+      assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
+      assertTrue(expressionString.contains("a_s as fieldA"));
+    }
   }
 
   @Test
   public void testDaemonStream() throws Exception {
-
-    DaemonStream stream;
     String expressionString;
 
     // Basic test
-    stream = new DaemonStream(StreamExpressionParser.parse("daemon(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), id=\"blah\", runInterval=\"1000\", queueSize=\"100\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("daemon(search(collection1,"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
-    assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
-    assertTrue(expressionString.contains("id=blah"));
-    assertTrue(expressionString.contains("queueSize=100"));
-    assertTrue(expressionString.contains("runInterval=1000"));
+    try (DaemonStream stream = new DaemonStream(StreamExpressionParser.parse("daemon(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), id=\"blah\", runInterval=\"1000\", queueSize=\"100\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("daemon(search(collection1,"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
+      assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
+      assertTrue(expressionString.contains("id=blah"));
+      assertTrue(expressionString.contains("queueSize=100"));
+      assertTrue(expressionString.contains("runInterval=1000"));
+    }
   }
 
   @Test
   public void testTopicStream() throws Exception {
 
-    TopicStream stream;
     String expressionString;
 
     // Basic test
-    stream = new TopicStream(StreamExpressionParser.parse("topic(collection2, collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", id=\"blah\", checkpointEvery=1000)"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("topic(collection2,collection1"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
-    assertTrue(expressionString.contains("id=blah"));
-    assertTrue(expressionString.contains("checkpointEvery=1000"));
+    try (TopicStream stream = new TopicStream(StreamExpressionParser.parse("topic(collection2, collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", id=\"blah\", checkpointEvery=1000)"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("topic(collection2,collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
+      assertTrue(expressionString.contains("id=blah"));
+      assertTrue(expressionString.contains("checkpointEvery=1000"));
+    }
   }
 
   @Test
   public void testStatsStream() throws Exception {
-
-    StatsStream stream;
     String expressionString;
     
     // Basic test
-    stream = new StatsStream(StreamExpressionParser.parse("stats(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", sum(a_i), avg(a_i), count(*), min(a_i), max(a_i))"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("stats(collection1,"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
-    assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
-    assertTrue(expressionString.contains("min(a_i)"));
-    assertTrue(expressionString.contains("max(a_i)"));
-    assertTrue(expressionString.contains("avg(a_i,false)"));
-    assertTrue(expressionString.contains("count(*)"));
-    assertTrue(expressionString.contains("sum(a_i)"));
-    
+    try (StatsStream stream = new StatsStream(StreamExpressionParser.parse("stats(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", sum(a_i), avg(a_i), count(*), min(a_i), max(a_i))"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("stats(collection1,"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("fl=\"id,a_s,a_i,a_f\""));
+      assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\""));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+    }
   }
 
   @Test
   public void testUniqueStream() throws Exception {
-
-    UniqueStream stream;
     String expressionString;
     
     // Basic test
-    stream = new UniqueStream(StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("unique(search(collection1"));
-    assertTrue(expressionString.contains("over=a_f"));
+    try (UniqueStream stream = new UniqueStream(StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("unique(search(collection1"));
+      assertTrue(expressionString.contains("over=a_f"));
+    }
   }
   
   @Test
   public void testMergeStream() throws Exception {
-
-    MergeStream stream;
     String expressionString;
     
     // Basic test
-    stream = new MergeStream(StreamExpressionParser.parse("merge("
+    try (MergeStream stream = new MergeStream(StreamExpressionParser.parse("merge("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f asc, a_s asc\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
-    assertTrue(expressionString.contains("q=\"id:(1 2)\""));
-    assertTrue(expressionString.contains("on=\"a_f asc,a_s asc\""));
+                              + "on=\"a_f asc, a_s asc\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
+      assertTrue(expressionString.contains("q=\"id:(1 2)\""));
+      assertTrue(expressionString.contains("on=\"a_f asc,a_s asc\""));
+    }
   }
   
   @Test
   public void testRankStream() throws Exception {
 
-    RankStream stream;
     String expressionString;
     
     // Basic test
-    stream = new RankStream(StreamExpressionParser.parse("top("
+    try (RankStream stream = new RankStream(StreamExpressionParser.parse("top("
                                               + "n=3,"
                                               + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc,a_i asc\"),"
-                                              + "sort=\"a_f asc, a_i asc\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("top(n=3,search(collection1"));
-    assertTrue(expressionString.contains("sort=\"a_f asc,a_i asc\""));
-    // find 2nd instance of sort
-    assertTrue(expressionString.substring(expressionString.indexOf("sort=\"a_f asc,a_i asc\"") + 1).contains("sort=\"a_f asc,a_i asc\""));
+                                              + "sort=\"a_f asc, a_i asc\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("top(n=3,search(collection1"));
+      assertTrue(expressionString.contains("sort=\"a_f asc,a_i asc\""));
+      // find 2nd instance of sort
+      assertTrue(expressionString.substring(expressionString.indexOf("sort=\"a_f asc,a_i asc\"") + 1).contains("sort=\"a_f asc,a_i asc\""));
+    }
   }
 
   @Test
   public void testReducerStream() throws Exception {
-
-    ReducerStream stream;
     String expressionString;
     
     // Basic test
-    stream = new ReducerStream(StreamExpressionParser.parse("reduce("
+    try (ReducerStream stream = new ReducerStream(StreamExpressionParser.parse("reduce("
                                                   + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc, a_f asc\"),"
-                                                  + "by=\"a_s\", group(sort=\"a_i desc\", n=\"5\"))"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("reduce(search(collection1"));
-    assertTrue(expressionString.contains("by=a_s"));
+                                                  + "by=\"a_s\", group(sort=\"a_i desc\", n=\"5\"))"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("reduce(search(collection1"));
+      assertTrue(expressionString.contains("by=a_s"));
+    }
   }
   
   @Test
@@ -238,22 +229,22 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
                                                                  + "fl=\"id,a_s,a_i,a_f\", "
                                                                  + "sort=\"a_f asc, a_i asc\"))");
     
-    UpdateStream updateStream = new UpdateStream(expression, factory);
-    String expressionString = updateStream.toExpression(factory).toString();
-    
-    assertTrue(expressionString.contains("update(collection2"));
-    assertTrue(expressionString.contains("batchSize=5"));
-    assertTrue(expressionString.contains("search(collection1"));
+    try (UpdateStream updateStream = new UpdateStream(expression, factory)) {
+      String expressionString = updateStream.toExpression(factory).toString();
+
+      assertTrue(expressionString.contains("update(collection2"));
+      assertTrue(expressionString.contains("batchSize=5"));
+      assertTrue(expressionString.contains("search(collection1"));
+    }
   }
   
   @Test
   public void testFacetStream() throws Exception {
 
-    FacetStream stream;
     String expressionString;
     
     // Basic test
-    stream = new FacetStream(StreamExpressionParser.parse("facet("
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
                                                         +   "collection1, "
                                                         +   "q=\"*:*\", "
                                                         +   "buckets=\"a_s\", "
@@ -264,68 +255,68 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
                                                         +   "max(a_i), max(a_f), "
                                                         +   "avg(a_i), avg(a_f), "
                                                         +   "count(*)"
-                                                        + ")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("facet(collection1"));
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("buckets=a_s"));
-    assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
-    assertTrue(expressionString.contains("bucketSizeLimit=100"));
-    assertTrue(expressionString.contains("sum(a_i)"));
-    assertTrue(expressionString.contains("sum(a_f)"));
-    assertTrue(expressionString.contains("min(a_i)"));
-    assertTrue(expressionString.contains("min(a_f)"));
-    assertTrue(expressionString.contains("max(a_i)"));
-    assertTrue(expressionString.contains("max(a_f)"));
-    assertTrue(expressionString.contains("avg(a_i,false)"));
-    assertTrue(expressionString.contains("avg(a_f,false)"));
-    assertTrue(expressionString.contains("count(*)"));
+                                                        + ")"), factory)){
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("facet(collection1"));
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("buckets=a_s"));
+      assertTrue(expressionString.contains("bucketSorts=\"sum(a_i) asc\""));
+      assertTrue(expressionString.contains("bucketSizeLimit=100"));
+      assertTrue(expressionString.contains("sum(a_i)"));
+      assertTrue(expressionString.contains("sum(a_f)"));
+      assertTrue(expressionString.contains("min(a_i)"));
+      assertTrue(expressionString.contains("min(a_f)"));
+      assertTrue(expressionString.contains("max(a_i)"));
+      assertTrue(expressionString.contains("max(a_f)"));
+      assertTrue(expressionString.contains("avg(a_i,false)"));
+      assertTrue(expressionString.contains("avg(a_f,false)"));
+      assertTrue(expressionString.contains("count(*)"));
+    }
   }
   
   @Test
   public void testJDBCStream() throws Exception {
-
-    JDBCStream stream;
     String expressionString;
     
     // Basic test
-    stream = new JDBCStream(StreamExpressionParser.parse("jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"ID asc\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("jdbc(connection=\"jdbc:hsqldb:mem:.\","));
-    assertTrue(expressionString.contains("sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\""));
-    assertTrue(expressionString.contains("sort=\"ID asc\""));
+    try (JDBCStream stream = new JDBCStream(StreamExpressionParser.parse("jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"ID asc\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("jdbc(connection=\"jdbc:hsqldb:mem:.\","));
+      assertTrue(expressionString.contains("sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\""));
+      assertTrue(expressionString.contains("sort=\"ID asc\""));
+    }
   }
 
   @Test 
   public void testIntersectStream() throws Exception {
-    IntersectStream stream;
     String expressionString;
     
     // Basic test
-    stream = new IntersectStream(StreamExpressionParser.parse("intersect("
+    try (IntersectStream stream = new IntersectStream(StreamExpressionParser.parse("intersect("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f, a_s\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
-    assertTrue(expressionString.contains("q=\"id:(1 2)\""));
-    assertTrue(expressionString.contains("on=\"a_f,a_s\""));
+                              + "on=\"a_f, a_s\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
+      assertTrue(expressionString.contains("q=\"id:(1 2)\""));
+      assertTrue(expressionString.contains("on=\"a_f,a_s\""));
+    }
   }
 
   @Test 
   public void testComplementStream() throws Exception {
-    ComplementStream stream;
     String expressionString;
     
     // Basic test
-    stream = new ComplementStream(StreamExpressionParser.parse("complement("
+    try (ComplementStream stream = new ComplementStream(StreamExpressionParser.parse("complement("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f, a_s\")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
-    assertTrue(expressionString.contains("q=\"id:(1 2)\""));
-    assertTrue(expressionString.contains("on=\"a_f,a_s\""));
+                              + "on=\"a_f, a_s\")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("q=\"id:(0 3 4)\""));
+      assertTrue(expressionString.contains("q=\"id:(1 2)\""));
+      assertTrue(expressionString.contains("on=\"a_f,a_s\""));
+    }
   }
   
   @Test
@@ -337,27 +328,30 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
     
     // Basic test
     String originalExpressionString = "search(collection1,fl=\"id,first\",sort=\"first asc\",q=\"presentTitles:\\\"chief, executive officer\\\" AND age:[36 TO *]\")";
-    CloudSolrStream firstStream = new CloudSolrStream(StreamExpressionParser.parse(originalExpressionString), factory);
-    String firstExpressionString = firstStream.toExpression(factory).toString();
-    
-    CloudSolrStream secondStream = new CloudSolrStream(StreamExpressionParser.parse(firstExpressionString), factory);
-    String secondExpressionString = secondStream.toExpression(factory).toString();
-    
-    assertTrue(firstExpressionString.contains("q=\"presentTitles:\\\"chief, executive officer\\\" AND age:[36 TO *]\""));
-    assertTrue(secondExpressionString.contains("q=\"presentTitles:\\\"chief, executive officer\\\" AND age:[36 TO *]\""));
+    try (CloudSolrStream firstStream = new CloudSolrStream(StreamExpressionParser.parse(originalExpressionString), factory)) {
+      String firstExpressionString = firstStream.toExpression(factory).toString();
+
+      try (CloudSolrStream secondStream = new CloudSolrStream(StreamExpressionParser.parse(firstExpressionString), factory)) {
+        String secondExpressionString = secondStream.toExpression(factory).toString();
+
+        assertTrue(firstExpressionString.contains("q=\"presentTitles:\\\"chief, executive officer\\\" AND age:[36 TO *]\""));
+        assertTrue(secondExpressionString.contains("q=\"presentTitles:\\\"chief, executive officer\\\" AND age:[36 TO *]\""));
+      }
+    }
   }
 
   @Test
   public void testFeaturesSelectionStream() throws Exception {
     String expr = "featuresSelection(collection1, q=\"*:*\", featureSet=\"first\", field=\"tv_text\", outcome=\"out_i\", numTerms=4, positiveLabel=2)";
-    FeaturesSelectionStream stream = new FeaturesSelectionStream(StreamExpressionParser.parse(expr), factory);
-    String expressionString = stream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("featureSet=first"));
-    assertTrue(expressionString.contains("field=tv_text"));
-    assertTrue(expressionString.contains("outcome=out_i"));
-    assertTrue(expressionString.contains("numTerms=4"));
-    assertTrue(expressionString.contains("positiveLabel=2"));
+    try (FeaturesSelectionStream stream = new FeaturesSelectionStream(StreamExpressionParser.parse(expr), factory)) {
+      String expressionString = stream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("featureSet=first"));
+      assertTrue(expressionString.contains("field=tv_text"));
+      assertTrue(expressionString.contains("outcome=out_i"));
+      assertTrue(expressionString.contains("numTerms=4"));
+      assertTrue(expressionString.contains("positiveLabel=2"));
+    }
   }
 
   @Test
@@ -370,15 +364,16 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase {
         "field=\"tv_text\", " +
         "outcome=\"out_i\", " +
         "maxIterations=100)";
-    TextLogitStream logitStream = new TextLogitStream(StreamExpressionParser.parse(expr), factory);
-    String expressionString = logitStream.toExpression(factory).toString();
-    assertTrue(expressionString.contains("q=\"*:*\""));
-    assertTrue(expressionString.contains("name=model"));
-    assertFalse(expressionString.contains("terms="));
-    assertTrue(expressionString.contains("featuresSelection("));
-    assertTrue(expressionString.contains("field=tv_text"));
-    assertTrue(expressionString.contains("outcome=out_i"));
-    assertTrue(expressionString.contains("maxIterations=100"));
+    try (TextLogitStream logitStream = new TextLogitStream(StreamExpressionParser.parse(expr), factory)) {
+      String expressionString = logitStream.toExpression(factory).toString();
+      assertTrue(expressionString.contains("q=\"*:*\""));
+      assertTrue(expressionString.contains("name=model"));
+      assertFalse(expressionString.contains("terms="));
+      assertTrue(expressionString.contains("featuresSelection("));
+      assertTrue(expressionString.contains("field=tv_text"));
+      assertTrue(expressionString.contains("outcome=out_i"));
+      assertTrue(expressionString.contains("maxIterations=100"));
+    }
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExplanationTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExplanationTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExplanationTest.java
index 91cab3d..c1c5369 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExplanationTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExplanationTest.java
@@ -69,126 +69,108 @@ public class StreamExpressionToExplanationTest extends LuceneTestCase {
     
   @Test
   public void testCloudSolrStream() throws Exception {
-
-    CloudSolrStream stream;
-    
     // Basic test
-    stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("search", explanation.getFunctionName());
-    Assert.assertEquals(CloudSolrStream.class.getName(), explanation.getImplementingClass());
-
+    try (CloudSolrStream stream = new CloudSolrStream(StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("search", explanation.getFunctionName());
+      Assert.assertEquals(CloudSolrStream.class.getName(), explanation.getImplementingClass());
+    }
   }
   
   @Test
   public void testSelectStream() throws Exception {
-
-    SelectStream stream;
-    
     // Basic test
-    stream = new SelectStream(StreamExpressionParser.parse("select(\"a_s as fieldA\", search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"))"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("select", explanation.getFunctionName());
-    Assert.assertEquals(SelectStream.class.getName(), explanation.getImplementingClass());    
+    try (SelectStream stream = new SelectStream(StreamExpressionParser.parse("select(\"a_s as fieldA\", search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"))"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("select", explanation.getFunctionName());
+      Assert.assertEquals(SelectStream.class.getName(), explanation.getImplementingClass());
+    }
   }
 
   @Test
   public void testDaemonStream() throws Exception {
-
-    DaemonStream stream;
-
     // Basic test
-    stream = new DaemonStream(StreamExpressionParser.parse("daemon(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), id=\"blah\", runInterval=\"1000\", queueSize=\"100\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("daemon", explanation.getFunctionName());
-    Assert.assertEquals(DaemonStream.class.getName(), explanation.getImplementingClass());
+    try (DaemonStream stream = new DaemonStream(StreamExpressionParser.parse("daemon(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), id=\"blah\", runInterval=\"1000\", queueSize=\"100\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("daemon", explanation.getFunctionName());
+      Assert.assertEquals(DaemonStream.class.getName(), explanation.getImplementingClass());
+    }
   }
 
   @Test
   public void testTopicStream() throws Exception {
-
-    TopicStream stream;
-
     // Basic test
-    stream = new TopicStream(StreamExpressionParser.parse("topic(collection2, collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", id=\"blah\", checkpointEvery=1000)"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("topic", explanation.getFunctionName());
-    Assert.assertEquals(TopicStream.class.getName(), explanation.getImplementingClass());
+    try (TopicStream stream = new TopicStream(StreamExpressionParser.parse("topic(collection2, collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", id=\"blah\", checkpointEvery=1000)"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("topic", explanation.getFunctionName());
+      Assert.assertEquals(TopicStream.class.getName(), explanation.getImplementingClass());
+    }
   }
 
 
   @Test
   public void testStatsStream() throws Exception {
-
-    StatsStream stream;
-    
     // Basic test
-    stream = new StatsStream(StreamExpressionParser.parse("stats(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", sum(a_i), avg(a_i), count(*), min(a_i), max(a_i))"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("stats", explanation.getFunctionName());
-    Assert.assertEquals(StatsStream.class.getName(), explanation.getImplementingClass());
-    
+    try (StatsStream stream = new StatsStream(StreamExpressionParser.parse("stats(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", sum(a_i), avg(a_i), count(*), min(a_i), max(a_i))"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("stats", explanation.getFunctionName());
+      Assert.assertEquals(StatsStream.class.getName(), explanation.getImplementingClass());
+    }
   }
 
   @Test
   public void testUniqueStream() throws Exception {
-
-    UniqueStream stream;
-    
     // Basic test
-    stream = new UniqueStream(StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("unique", explanation.getFunctionName());
-    Assert.assertEquals(UniqueStream.class.getName(), explanation.getImplementingClass());
+    try (UniqueStream stream = new UniqueStream(StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("unique", explanation.getFunctionName());
+      Assert.assertEquals(UniqueStream.class.getName(), explanation.getImplementingClass());
+    }
   }
   
   @Test
   public void testMergeStream() throws Exception {
-
-    MergeStream stream;
-    
     // Basic test
-    stream = new MergeStream(StreamExpressionParser.parse("merge("
+    try (MergeStream stream = new MergeStream(StreamExpressionParser.parse("merge("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f asc, a_s asc\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("merge", explanation.getFunctionName());
-    Assert.assertEquals(MergeStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(2, ((StreamExplanation)explanation).getChildren().size());
+                              + "on=\"a_f asc, a_s asc\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("merge", explanation.getFunctionName());
+      Assert.assertEquals(MergeStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(2, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
   
   @Test
   public void testRankStream() throws Exception {
-
-    RankStream stream;
     String expressionString;
     
     // Basic test
-    stream = new RankStream(StreamExpressionParser.parse("top("
+    try (RankStream stream = new RankStream(StreamExpressionParser.parse("top("
                                               + "n=3,"
                                               + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc,a_i asc\"),"
-                                              + "sort=\"a_f asc, a_i asc\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("top", explanation.getFunctionName());
-    Assert.assertEquals(RankStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(1, ((StreamExplanation)explanation).getChildren().size());
+                                              + "sort=\"a_f asc, a_i asc\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("top", explanation.getFunctionName());
+      Assert.assertEquals(RankStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(1, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
 
   @Test
   public void testReducerStream() throws Exception {
-
-    ReducerStream stream;
     String expressionString;
     
     // Basic test
-    stream = new ReducerStream(StreamExpressionParser.parse("reduce("
+    try (ReducerStream stream = new ReducerStream(StreamExpressionParser.parse("reduce("
                                                   + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc, a_f asc\"),"
-                                                  + "by=\"a_s\", group(sort=\"a_i desc\", n=\"5\"))"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("reduce", explanation.getFunctionName());
-    Assert.assertEquals(ReducerStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(1, ((StreamExplanation)explanation).getChildren().size());
+                                                  + "by=\"a_s\", group(sort=\"a_i desc\", n=\"5\"))"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("reduce", explanation.getFunctionName());
+      Assert.assertEquals(ReducerStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(1, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
   
   @Test
@@ -202,26 +184,25 @@ public class StreamExpressionToExplanationTest extends LuceneTestCase {
                                                                  + "fl=\"id,a_s,a_i,a_f\", "
                                                                  + "sort=\"a_f asc, a_i asc\"))");
     
-    UpdateStream updateStream = new UpdateStream(expression, factory);
-    Explanation explanation = updateStream.toExplanation(factory);
-    Assert.assertEquals("solr (collection2)", explanation.getFunctionName());
-    Assert.assertEquals("Solr/Lucene", explanation.getImplementingClass());
-    
-    StreamExplanation updateExplanation = (StreamExplanation)explanation;
-    Assert.assertEquals(1, updateExplanation.getChildren().size());
-    Assert.assertEquals("update", updateExplanation.getChildren().get(0).getFunctionName());
-    Assert.assertEquals(UpdateStream.class.getName(), updateExplanation.getChildren().get(0).getImplementingClass());
+    try (UpdateStream updateStream = new UpdateStream(expression, factory)) {
+      Explanation explanation = updateStream.toExplanation(factory);
+      Assert.assertEquals("solr (collection2)", explanation.getFunctionName());
+      Assert.assertEquals("Solr/Lucene", explanation.getImplementingClass());
+
+      StreamExplanation updateExplanation = (StreamExplanation) explanation;
+      Assert.assertEquals(1, updateExplanation.getChildren().size());
+      Assert.assertEquals("update", updateExplanation.getChildren().get(0).getFunctionName());
+      Assert.assertEquals(UpdateStream.class.getName(), updateExplanation.getChildren().get(0).getImplementingClass());
+    }
     
   }
   
   @Test
   public void testFacetStream() throws Exception {
-
-    FacetStream stream;
     String expressionString;
     
     // Basic test
-    stream = new FacetStream(StreamExpressionParser.parse("facet("
+    try (FacetStream stream = new FacetStream(StreamExpressionParser.parse("facet("
                                                         +   "collection1, "
                                                         +   "q=\"*:*\", "
                                                         +   "buckets=\"a_s\", "
@@ -232,57 +213,57 @@ public class StreamExpressionToExplanationTest extends LuceneTestCase {
                                                         +   "max(a_i), max(a_f), "
                                                         +   "avg(a_i), avg(a_f), "
                                                         +   "count(*)"
-                                                        + ")"), factory);
-    expressionString = stream.toExpression(factory).toString();
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("facet", explanation.getFunctionName());
-    Assert.assertEquals(FacetStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(1, ((StreamExplanation)explanation).getChildren().size());
+                                                        + ")"), factory)) {
+      expressionString = stream.toExpression(factory).toString();
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("facet", explanation.getFunctionName());
+      Assert.assertEquals(FacetStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(1, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
   
   @Test
   public void testJDBCStream() throws Exception {
-
-    JDBCStream stream;
     String expressionString;
     
     // Basic test
-    stream = new JDBCStream(StreamExpressionParser.parse("jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"ID asc\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("jdbc", explanation.getFunctionName());
-    Assert.assertEquals(JDBCStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(1, ((StreamExplanation)explanation).getChildren().size());
+    try (JDBCStream stream = new JDBCStream(StreamExpressionParser.parse("jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"ID asc\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("jdbc", explanation.getFunctionName());
+      Assert.assertEquals(JDBCStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(1, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
 
   @Test 
   public void testIntersectStream() throws Exception {
-    IntersectStream stream;
     String expressionString;
     
     // Basic test
-    stream = new IntersectStream(StreamExpressionParser.parse("intersect("
+    try (IntersectStream stream = new IntersectStream(StreamExpressionParser.parse("intersect("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f, a_s\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("intersect", explanation.getFunctionName());
-    Assert.assertEquals(IntersectStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(2, ((StreamExplanation)explanation).getChildren().size());
+                              + "on=\"a_f, a_s\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("intersect", explanation.getFunctionName());
+      Assert.assertEquals(IntersectStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(2, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
 
   @Test 
   public void testComplementStream() throws Exception {
-    ComplementStream stream;
     String expressionString;
     
     // Basic test
-    stream = new ComplementStream(StreamExpressionParser.parse("complement("
+    try (ComplementStream stream = new ComplementStream(StreamExpressionParser.parse("complement("
                               + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
                               + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\"),"
-                              + "on=\"a_f, a_s\")"), factory);
-    Explanation explanation = stream.toExplanation(factory);
-    Assert.assertEquals("complement", explanation.getFunctionName());
-    Assert.assertEquals(ComplementStream.class.getName(), explanation.getImplementingClass());
-    Assert.assertEquals(2, ((StreamExplanation)explanation).getChildren().size());
+                              + "on=\"a_f, a_s\")"), factory)) {
+      Explanation explanation = stream.toExplanation(factory);
+      Assert.assertEquals("complement", explanation.getFunctionName());
+      Assert.assertEquals(ComplementStream.class.getName(), explanation.getImplementingClass());
+      Assert.assertEquals(2, ((StreamExplanation) explanation).getChildren().size());
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
index 0de3aa0..9ee6688 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java
@@ -157,6 +157,7 @@ public void testSpacesInParams() throws Exception {
   //a value in the field list.
 
   CloudSolrStream stream = new CloudSolrStream("", "collection1", sParams);
+  stream.close();
 }
 
 @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
index c813ea2a..cda751d 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/NoOpResponseParserTest.java
@@ -20,10 +20,12 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
+import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.input.ReaderInputStream;
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
@@ -36,7 +38,6 @@ import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrResourceLoader;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -49,7 +50,7 @@ import org.junit.Test;
 public class NoOpResponseParserTest extends SolrJettyTestBase {
 
   private static InputStream getResponse() throws IOException {
-    return new SolrResourceLoader().openResource("solrj/sampleRangeFacetResponse.xml");
+    return new ReaderInputStream(new StringReader("NO-OP test response"), StandardCharsets.UTF_8);
   }
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/response/QueryResponseTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/QueryResponseTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/QueryResponseTest.java
index 7a69815..a567116 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/QueryResponseTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/QueryResponseTest.java
@@ -43,11 +43,15 @@ public class QueryResponseTest extends LuceneTestCase {
   @Test
   public void testRangeFacets() throws Exception {
     XMLResponseParser parser = new XMLResponseParser();
-    InputStream is = new SolrResourceLoader().openResource("solrj/sampleRangeFacetResponse.xml");
-    assertNotNull(is);
-    Reader in = new InputStreamReader(is, StandardCharsets.UTF_8);
-    NamedList<Object> response = parser.processResponse(in);
-    in.close();
+    NamedList<Object> response = null;
+    try (SolrResourceLoader loader = new SolrResourceLoader();
+         InputStream is = loader.openResource("solrj/sampleRangeFacetResponse.xml")) {
+      assertNotNull(is);
+
+      try (Reader in = new InputStreamReader(is, StandardCharsets.UTF_8)) {
+        response = parser.processResponse(in);
+      }
+    }
 
     QueryResponse qr = new QueryResponse(response, null);
     Assert.assertNotNull(qr);
@@ -100,11 +104,14 @@ public class QueryResponseTest extends LuceneTestCase {
   @Test
   public void testGroupResponse() throws Exception {
     XMLResponseParser parser = new XMLResponseParser();
-    InputStream is = new SolrResourceLoader().openResource("solrj/sampleGroupResponse.xml");
-    assertNotNull(is);
-    Reader in = new InputStreamReader(is, StandardCharsets.UTF_8);
-    NamedList<Object> response = parser.processResponse(in);
-    in.close();
+    NamedList<Object> response = null;
+    try (SolrResourceLoader loader = new SolrResourceLoader();
+         InputStream is = loader.openResource("solrj/sampleGroupResponse.xml")) {
+      assertNotNull(is);
+      try (Reader in = new InputStreamReader(is, StandardCharsets.UTF_8)) {
+        response = parser.processResponse(in);
+      }
+    }
 
     QueryResponse qr = new QueryResponse(response, null);
     assertNotNull(qr);
@@ -202,11 +209,15 @@ public class QueryResponseTest extends LuceneTestCase {
   @Test
   public void testSimpleGroupResponse() throws Exception {
     XMLResponseParser parser = new XMLResponseParser();
-    InputStream is = new SolrResourceLoader().openResource("solrj/sampleSimpleGroupResponse.xml");
-    assertNotNull(is);
-    Reader in = new InputStreamReader(is, StandardCharsets.UTF_8);
-    NamedList<Object> response = parser.processResponse(in);
-    in.close();
+    NamedList<Object> response = null;
+
+    try (SolrResourceLoader loader = new SolrResourceLoader();
+         InputStream is = loader.openResource("solrj/sampleSimpleGroupResponse.xml")) {
+      assertNotNull(is);
+      try (Reader in = new InputStreamReader(is, StandardCharsets.UTF_8)) {
+        response = parser.processResponse(in);
+      }
+    }
 
     QueryResponse qr = new QueryResponse(response, null);
     assertNotNull(qr);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/61646438/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestClusteringResponse.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestClusteringResponse.java b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestClusteringResponse.java
index 7e789d1..75bb0d3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestClusteringResponse.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/response/TestClusteringResponse.java
@@ -37,13 +37,16 @@ public class TestClusteringResponse extends SolrJettyTestBase {
   @Test
   public void testClusteringResponse() throws Exception {
     XMLResponseParser parser = new XMLResponseParser();
-    /*Load a simple XML with the clustering response encoded in an XML format*/
-    InputStream is = new SolrResourceLoader().openResource("solrj/sampleClusteringResponse.xml");
-    assertNotNull(is);
-    Reader in = new InputStreamReader(is, StandardCharsets.UTF_8);
-    NamedList<Object> response = parser.processResponse(in);
-    in.close();
+    NamedList<Object> response = null;
 
+    /*Load a simple XML with the clustering response encoded in an XML format*/
+    try (SolrResourceLoader loader = new SolrResourceLoader();
+         InputStream is = loader.openResource("solrj/sampleClusteringResponse.xml")) {
+      assertNotNull(is);
+      try (Reader in = new InputStreamReader(is, StandardCharsets.UTF_8)) {
+        response = parser.processResponse(in);
+      }
+    }
     QueryResponse qr = new QueryResponse(response, null);
     ClusteringResponse clusteringResponse = qr.getClusteringResponse();
     List<Cluster> clusters = clusteringResponse.getClusters();