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 2019/06/04 01:03:31 UTC

[lucene-solr] branch branch_8x updated: SOLR-8346: Upgrade Zookeeper to version 3.5.5

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

erick pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new e890980  SOLR-8346: Upgrade Zookeeper to version 3.5.5
e890980 is described below

commit e8909803110f8466513a04841611baf03386485c
Author: Erick Erickson <Er...@gmail.com>
AuthorDate: Mon Jun 3 17:50:35 2019 -0700

    SOLR-8346: Upgrade Zookeeper to version 3.5.5
    
    (cherry picked from commit 7ebeab71f4b56d008835f0040468a07632024950)
---
 lucene/ivy-versions.properties                     |   5 +-
 solr/CHANGES.txt                                   |  11 +-
 .../handler/dataimport/TestZKPropertiesWriter.java |   5 +-
 .../java/org/apache/solr/cloud/SolrZkServer.java   |  13 +-
 .../core/src/java/org/apache/solr/cloud/ZkCLI.java |   2 +-
 .../src/java/org/apache/solr/core/ZkContainer.java |   3 +-
 .../solr/handler/admin/ZookeeperStatusHandler.java |  39 +++-
 .../apache/solr/cloud/ConnectionManagerTest.java   |  10 +-
 .../apache/solr/cloud/DistributedQueueTest.java    |   2 +-
 .../org/apache/solr/cloud/LeaderElectionTest.java  | 105 +++++------
 .../OutOfBoxZkACLAndCredentialsProvidersTest.java  |  23 ++-
 ...OverriddenZkACLAndCredentialsProvidersTest.java |   7 +-
 .../test/org/apache/solr/cloud/OverseerTest.java   | 193 ++++++++++----------
 .../apache/solr/cloud/SaslZkACLProviderTest.java   |  20 +-
 .../org/apache/solr/cloud/SolrXmlInZkTest.java     |   4 +-
 .../solr/cloud/TestConfigSetsAPIZkFailure.java     |  14 +-
 .../org/apache/solr/cloud/TestDistributedMap.java  |   2 +-
 .../solr/cloud/TestLeaderElectionZkExpiry.java     |   2 +-
 .../test/org/apache/solr/cloud/TestZkChroot.java   |   4 +-
 .../VMParamsZkACLAndCredentialsProvidersTest.java  |   7 +-
 .../src/test/org/apache/solr/cloud/ZkCLITest.java  |  73 ++++----
 .../org/apache/solr/cloud/ZkControllerTest.java    |   9 +-
 .../org/apache/solr/cloud/ZkSolrClientTest.java    |   7 +-
 .../solr/cloud/api/collections/AssignTest.java     |   5 +-
 .../sim/TestSimDistribStateManager.java            |   2 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java     |   9 +-
 .../solr/cloud/overseer/ZkStateWriterTest.java     |  11 +-
 .../handler/admin/ZookeeperStatusHandlerTest.java  |   6 +-
 .../solr/schema/TestManagedSchemaThreadSafety.java |   2 +-
 solr/licenses/zookeeper-3.4.14.jar.sha1            |   1 -
 solr/licenses/zookeeper-3.5.5.jar.sha1             |   1 +
 solr/licenses/zookeeper-jute-3.5.5.jar.sha1        |   1 +
 solr/licenses/zookeeper-jute-LICENSE-ASL.txt       | 202 +++++++++++++++++++++
 solr/licenses/zookeeper-jute-NOTICE.txt            |   5 +
 .../setting-up-an-external-zookeeper-ensemble.adoc |   6 +
 solr/solrj/ivy.xml                                 |   1 +
 .../org/apache/solr/common/cloud/SolrZkClient.java |   1 +
 .../apache/solr/common/cloud/SolrZkClientTest.java |   3 +-
 .../solr/common/cloud/TestZkConfigManager.java     |   2 +-
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |   3 +
 .../solr/cloud/AbstractDistribZkTestBase.java      |  38 ++--
 .../org/apache/solr/cloud/AbstractZkTestCase.java  |   5 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   2 +-
 .../java/org/apache/solr/cloud/ZkTestServer.java   |  30 +--
 44 files changed, 581 insertions(+), 315 deletions(-)

diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 7767b7c..0045d3b 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -196,7 +196,10 @@ org.apache.velocity.tools.version = 3.0
 /org.apache.velocity/velocity-engine-core = 2.0
 
 /org.apache.xmlbeans/xmlbeans = 3.0.1
-/org.apache.zookeeper/zookeeper = 3.4.14
+
+org.apache.zookeeper.version = 3.5.5
+/org.apache.zookeeper/zookeeper = ${org.apache.zookeeper.version}
+/org.apache.zookeeper/zookeeper-jute = ${org.apache.zookeeper.version}
 
 # v1.6.0-alpha.5 of asciidoctor-ant includes asciidoctorj-pdf 1.5.0-alpha.16,
 # which is the same as asciidoctor-pdf 1.5.0-alpha.16
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a2d4d29..9c0ac8f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -25,9 +25,18 @@ Versions of Major Components
 Apache Tika 1.19.1
 Carrot2 3.16.0
 Velocity 2.0 and Velocity Tools 3.0
-Apache ZooKeeper 3.4.14
+Apache ZooKeeper 3.5.5
 Jetty 9.4.14.v20181114
 
+Upgrade Notes
+----------------------
+
+* SOLR-8346: Upgrade ZooKeeper to 3.5.5. ZooKeeper 3.5.5 introduces more security features. Include in your zoo.cfg
+  file at minimum the following:
+  4lw.commands.whitelist=mntr,conf,ruok
+  You can use
+  4lw.commands.whitelist=* to enable all ZooKeeper "4 letter commands".
+  (Erick Erickson)
 
 New Features
 ----------------------
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java
index 14c9e98..1d6f591 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestZKPropertiesWriter.java
@@ -18,6 +18,7 @@ package org.apache.solr.handler.dataimport;
 
 import java.lang.invoke.MethodHandles;
 
+import java.nio.file.Path;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
@@ -45,7 +46,7 @@ public class TestZKPropertiesWriter extends AbstractDataImportHandlerTestCase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected static ZkTestServer zkServer;
 
-  protected static String zkDir;
+  protected static Path zkDir;
 
   private static CoreContainer cc;
 
@@ -53,7 +54,7 @@ public class TestZKPropertiesWriter extends AbstractDataImportHandlerTestCase {
 
   @BeforeClass
   public static void dihZk_beforeClass() throws Exception {
-    zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    zkDir = createTempDir("zkData");
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java b/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
index 664b541..3d098aa 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
@@ -41,7 +41,9 @@ import java.util.regex.Pattern;
 
 public class SolrZkServer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+
+  public static final String ZK_WHITELIST_PROPERTY = "zookeeper.4lw.commands.whitelist";
+
   String zkRun;
   String zkHost;
 
@@ -51,10 +53,10 @@ public class SolrZkServer {
 
   private Thread zkThread;  // the thread running a zookeeper server, only if zkRun is set
 
-  private String dataHome;
+  private File dataHome;
   private String confHome;
 
-  public SolrZkServer(String zkRun, String zkHost, String dataHome, String confHome, int solrPort) {
+  public SolrZkServer(String zkRun, String zkHost, File dataHome, String confHome, int solrPort) {
     this.zkRun = zkRun;
     this.zkHost = zkHost;
     this.dataHome = dataHome;
@@ -102,6 +104,9 @@ public class SolrZkServer {
   public void start() {
     if (zkRun == null) return;
 
+    if (System.getProperty(ZK_WHITELIST_PROPERTY) == null) {
+      System.setProperty(ZK_WHITELIST_PROPERTY, "ruok, mntr, conf");
+    }
     zkThread = new Thread() {
       @Override
       public void run() {
@@ -293,7 +298,7 @@ class SolrZkServerProps extends QuorumPeerConfig {
 
 
 
-  public void setDataDir(String dataDir) {
+  public void setDataDir(File dataDir) {
     this.dataDir = dataDir;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
index 2df87a0..a751114 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
@@ -187,7 +187,7 @@ public class ZkCLI {
       
       SolrZkServer zkServer = null;
       if (solrPort != null) {
-        zkServer = new SolrZkServer("true", null, solrHome + "/zoo_data",
+        zkServer = new SolrZkServer("true", null, new File(solrHome, "/zoo_data"),
             solrHome, Integer.parseInt(solrPort));
         zkServer.parseConfig();
         zkServer.start();
diff --git a/solr/core/src/java/org/apache/solr/core/ZkContainer.java b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
index ae9c54a..c8217c6 100644
--- a/solr/core/src/java/org/apache/solr/core/ZkContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/ZkContainer.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.core;
 
+import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
@@ -81,7 +82,7 @@ public class ZkContainer {
     if (zkRun != null) {
       String zkDataHome = System.getProperty("zkServerDataDir", Paths.get(solrHome).resolve("zoo_data").toString());
       String zkConfHome = System.getProperty("zkServerConfDir", solrHome);
-      zkServer = new SolrZkServer(stripChroot(zkRun), stripChroot(config.getZkHost()), zkDataHome, zkConfHome, config.getSolrHostPort());
+      zkServer = new SolrZkServer(stripChroot(zkRun), stripChroot(config.getZkHost()), new File(zkDataHome), zkConfHome, config.getSolrHostPort());
       zkServer.parseConfig();
       zkServer.start();
       
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
index 1f3a504..7afd6bf 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperStatusHandler.java
@@ -94,6 +94,8 @@ public final class ZookeeperStatusHandler extends RequestHandlerBase {
     int reportedFollowers = 0;
     int leaders = 0;
     List<String> errors = new ArrayList<>();
+    zkStatus.put("ensembleSize", zookeepers.size());
+    zkStatus.put("zkHost", zkHost);
     for (String zk : zookeepers) {
       try {
         Map<String, Object> stat = monitorZookeeper(zk);
@@ -113,14 +115,14 @@ public final class ZookeeperStatusHandler extends RequestHandlerBase {
       } catch (SolrException se) {
         log.warn("Failed talking to zookeeper" + zk, se);
         errors.add(se.getMessage());
+        zkStatus.put("errors", errors);
         Map<String, Object> stat = new HashMap<>();
         stat.put("host", zk);
         stat.put("ok", false);
-        details.add(stat);
+        zkStatus.put("status", STATUS_YELLOW);
+        return zkStatus;
       }       
     }
-    zkStatus.put("ensembleSize", zookeepers.size());
-    zkStatus.put("zkHost", zkHost);
     zkStatus.put("details", details);
     if (followers+leaders > 0 && standalone > 0) {
       status = STATUS_RED;
@@ -176,17 +178,37 @@ public final class ZookeeperStatusHandler extends RequestHandlerBase {
     return zkStatus;
   }
 
-  private Map<String, Object> monitorZookeeper(String zkHostPort) {
-    List<String> lines = getZkRawResponse(zkHostPort, "mntr");
+  private Map<String, Object> monitorZookeeper(String zkHostPort) throws SolrException {
     Map<String, Object> obj = new HashMap<>();
     obj.put("host", zkHostPort);
-    obj.put("ok", "imok".equals(getZkRawResponse(zkHostPort, "ruok").get(0)));
+    List<String> lines = getZkRawResponse(zkHostPort, "ruok");
+    boolean ok = "imok".equals(lines.get(0));
+    if (ok == false) {
+      log.warn("Check 4lw.commands.whitelist setting in zookeeper configuration file, ZK response {}", lines.get(0));
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, lines.get(0) + " Check 4lw.commands.whitelist setting in zookeeper configuration file.");
+    }
+    obj.put("ok", ok);
+    lines = getZkRawResponse(zkHostPort, "mntr");
+    String[] parts;
     for (String line : lines) {
-      obj.put(line.split("\t")[0], line.split("\t")[1]);
+      parts = line.split("\t");
+      if (parts.length >= 2) {
+        obj.put(parts[0], parts[1]);
+      } else {
+        log.warn("Check 4lw.commands.whitelist setting in zookeeper configuration file, ZK response {}", line);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, line + " Check 4lw.commands.whitelist setting in zookeeper configuration file.");
+      }
     }
     lines = getZkRawResponse(zkHostPort, "conf");
+
     for (String line : lines) {
-      obj.put(line.split("=")[0], line.split("=")[1]);
+      parts = line.split("=");
+      if (parts.length >= 2) {
+        obj.put(parts[0], parts[1]);
+      } else {
+        log.warn("Check 4lw.commands.whitelist setting in zookeeper configuration file, ZK response {}", line);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, line + " Check 4lw.commands.whitelist setting in zookeeper configuration file.");
+      }
     }
     return obj;
   }
@@ -204,6 +226,7 @@ public final class ZookeeperStatusHandler extends RequestHandlerBase {
     if (hostPort.length > 1) {
       port = Integer.parseInt(hostPort[1]);
     }
+
     try (
         Socket socket = new Socket(host, port);
         Writer writer = new OutputStreamWriter(socket.getOutputStream(), "utf-8");
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
index 3fa2108..3993d38 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr.cloud;
 
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeoutException;
@@ -43,8 +44,7 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
   public void testConnectionManager() throws Exception {
     
     // setup a SolrZkClient to do some getBaseUrlForNodeName testing
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
-    
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = new ZkTestServer(zkDir);
     try {
       server.run();
@@ -73,8 +73,7 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
   public void testLikelyExpired() throws Exception {
 
     // setup a SolrZkClient to do some getBaseUrlForNodeName testing
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
-
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = new ZkTestServer(zkDir);
     try {
       server.run();
@@ -117,8 +116,7 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
     ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("connectionManagerTest"));
     
     // setup a SolrZkClient to do some getBaseUrlForNodeName testing
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
-
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = new ZkTestServer(zkDir);
     try {
       server.run();
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
index 7b9ae11..7108945 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
@@ -341,7 +341,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
 
   protected void setupZk() throws Exception {
     System.setProperty("zkClientTimeout", "8000");
-    zkServer = new ZkTestServer(createTempDir("zkData").toFile().getAbsolutePath());
+    zkServer = new ZkTestServer(createTempDir("zkData"));
     zkServer.run();
     System.setProperty("zkHost", zkServer.getZkAddress());
     zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractZkTestCase.TIMEOUT);
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
index 5586874..f7ff7dc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -48,30 +49,30 @@ import org.slf4j.LoggerFactory;
 public class LeaderElectionTest extends SolrTestCaseJ4 {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+
   static final int TIMEOUT = 30000;
   private ZkTestServer server;
   private SolrZkClient zkClient;
   private ZkStateReader zkStateReader;
   private Map<Integer,Thread> seqToThread;
-  
+
   private volatile boolean stopStress = false;
-  
+
   @BeforeClass
   public static void beforeClass() {
 
   }
-  
+
   @AfterClass
   public static void afterClass() {
 
   }
-  
+
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();;
-    
+    Path zkDir = createTempDir("zkData");
+
     server = new ZkTestServer(zkDir);
     server.setTheTickTime(1000);
     server.run();
@@ -179,7 +180,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
         es.close();
         return;
       }
-        
+
       while (!stop) {
         try {
           Thread.sleep(100);
@@ -187,9 +188,9 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
           return;
         }
       }
-      
+
     }
-    
+
     public void close() {
       es.close();
       this.stop = true;
@@ -283,69 +284,69 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
 
   @Test
   public void testElection() throws Exception {
-    
+
     List<ClientThread> threads = new ArrayList<>();
-    
+
     for (int i = 0; i < 15; i++) {
       ClientThread thread = new ClientThread("shard1", i);
       threads.add(thread);
     }
     try {
       startAndJoinElection(threads);
-      
+
       int leaderThread = getLeaderThread();
-      
+
       // whoever the leader is, should be the n_0 seq
       assertEquals(0, threads.get(leaderThread).seq);
 
       // kill n_0, 1, 3 and 4
       ((ClientThread) seqToThread.get(0)).close();
-      
+
       waitForLeader(threads, 1);
-      
+
       leaderThread = getLeaderThread();
-      
+
       // whoever the leader is, should be the n_1 seq
-      
+
       assertEquals(1, threads.get(leaderThread).seq);
-      
+
       ((ClientThread) seqToThread.get(4)).close();
       ((ClientThread) seqToThread.get(1)).close();
       ((ClientThread) seqToThread.get(3)).close();
-      
+
       // whoever the leader is, should be the n_2 seq
-      
+
       waitForLeader(threads, 2);
-      
+
       leaderThread = getLeaderThread();
       assertEquals(2, threads.get(leaderThread).seq);
-      
+
       // kill n_5, 2, 6, 7, and 8
       ((ClientThread) seqToThread.get(5)).close();
       ((ClientThread) seqToThread.get(2)).close();
       ((ClientThread) seqToThread.get(6)).close();
       ((ClientThread) seqToThread.get(7)).close();
       ((ClientThread) seqToThread.get(8)).close();
-      
+
       waitForLeader(threads, 9);
       leaderThread = getLeaderThread();
-      
+
       // whoever the leader is, should be the n_9 seq
       assertEquals(9, threads.get(leaderThread).seq);
-      
+
     } finally {
       // cleanup any threads still running
       for (ClientThread thread : threads) {
         thread.close();
         thread.interrupt();
-        
+
       }
-      
+
       for (Thread thread : threads) {
         thread.join();
       }
     }
-    
+
   }
 
   @Test
@@ -415,21 +416,21 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
     String leaderUrl = getLeaderUrl("collection1", "shard1");
     return Integer.parseInt(leaderUrl.replaceAll("/", ""));
   }
-  
+
   @Test
   public void testStressElection() throws Exception {
     final ScheduledExecutorService scheduler = Executors
         .newScheduledThreadPool(15, new DefaultSolrThreadFactory("stressElection"));
     final List<ClientThread> threads = Collections
         .synchronizedList(new ArrayList<ClientThread>());
-    
+
     // start with a leader
     ClientThread thread1 = null;
     thread1 = new ClientThread("shard1", 0);
     threads.add(thread1);
     scheduler.schedule(thread1, 0, TimeUnit.MILLISECONDS);
-    
-    
+
+
 
     Thread scheduleThread = new Thread() {
       @Override
@@ -450,11 +451,11 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
         }
       }
     };
-    
+
     Thread killThread = new Thread() {
       @Override
       public void run() {
-        
+
         while (!stopStress) {
           try {
             int j;
@@ -475,11 +476,11 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
         }
       }
     };
-    
+
     Thread connLossThread = new Thread() {
       @Override
       public void run() {
-        
+
         while (!stopStress) {
           try {
             Thread.sleep(50);
@@ -495,49 +496,49 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
               e.printStackTrace();
             }
             Thread.sleep(500);
-            
+
           } catch (Exception e) {
-            
+
           }
         }
       }
     };
-    
+
     scheduleThread.start();
     connLossThread.start();
     killThread.start();
-    
+
     Thread.sleep(4000);
-    
+
     stopStress = true;
-    
+
     scheduleThread.interrupt();
     connLossThread.interrupt();
     killThread.interrupt();
-    
+
     scheduleThread.join();
     scheduler.shutdownNow();
-    
+
     connLossThread.join();
     killThread.join();
-    
+
     int seq = threads.get(getLeaderThread()).getSeq();
-    
+
     // we have a leader we know, TODO: lets check some other things
-    
+
     // cleanup any threads still running
     for (ClientThread thread : threads) {
       thread.es.zkClient.getSolrZooKeeper().close();
       thread.close();
     }
-    
+
     for (Thread thread : threads) {
       thread.join();
     }
 
-    
+
   }
-  
+
   @Override
   public void tearDown() throws Exception {
     zkClient.close();
@@ -545,7 +546,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
     server.shutdown();
     super.tearDown();
   }
-  
+
   private void printLayout() throws Exception {
     zkClient.printLayoutToStdOut();
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
index 0884576..873480f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OutOfBoxZkACLAndCredentialsProvidersTest.java
@@ -16,9 +16,9 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -42,8 +42,8 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
   private static final Charset DATA_ENCODING = Charset.forName("UTF-8");
   
   protected ZkTestServer zkServer;
-  
-  protected String zkDir;
+
+  protected Path zkDir;
   
   @BeforeClass
   public static void beforeClass() {
@@ -60,9 +60,8 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     super.setUp();
     log.info("####SETUP_START " + getTestName());
     createTempDir();
-    
-    zkDir = createTempDir() + File.separator
-        + "zookeeper/server1/data";
+
+    zkDir = createTempDir().resolve("zookeeper/server1/data");
     log.info("ZooKeeper dataDir:" + zkDir);
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
@@ -126,11 +125,19 @@ public class OutOfBoxZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     if (log.isInfoEnabled()) {
       log.info("Verifying " + path);
     }
-    assertEquals("Path " + path + " does not have OPEN_ACL_UNSAFE", ZooDefs.Ids.OPEN_ACL_UNSAFE, acls);
+    if (ZooDefs.CONFIG_NODE.equals(path)) {
+      // Treat this node specially, from the ZK docs:
+      // The dynamic configuration is stored in a special znode ZooDefs.CONFIG_NODE = /zookeeper/config.
+      // This node by default is read only for all users, except super user and
+      // users that's explicitly configured for write access.
+      assertEquals("Path " + path + " does not have READ_ACL_UNSAFE", ZooDefs.Ids.READ_ACL_UNSAFE, acls);
+    } else {
+      assertEquals("Path " + path + " does not have OPEN_ACL_UNSAFE", ZooDefs.Ids.OPEN_ACL_UNSAFE, acls);
+    }
     verifiedList.add(path);
     List<String> children = zkClient.getChildren(path, null, false);
     for (String child : children) {
-      assertOpenACLUnsafeAllover(zkClient, path + ((path.endsWith("/"))?"":"/") + child, verifiedList);
+      assertOpenACLUnsafeAllover(zkClient, path + ((path.endsWith("/")) ? "" : "/") + child, verifiedList);
     }
   }
   
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
index 9596371..dfbbd95 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java
@@ -33,10 +33,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -49,7 +49,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
   
   protected ZkTestServer zkServer;
   
-  protected String zkDir;
+  protected Path zkDir;
   
   @BeforeClass
   public static void beforeClass() {
@@ -67,8 +67,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     log.info("####SETUP_START " + getTestName());
     createTempDir();
     
-    zkDir =createTempDir() + File.separator
-        + "zookeeper/server1/data";
+    zkDir =createTempDir().resolve("zookeeper/server1/data");
     log.info("ZooKeeper dataDir:" + zkDir);
     zkServer = new ZkTestServer(zkDir);
     zkServer.run(false);
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 5a2bbb9..ed11694 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -27,6 +27,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -115,10 +116,10 @@ public class OverseerTest extends SolrTestCaseJ4 {
   private static ZkTestServer server;
 
   private static SolrZkClient zkClient;
-  
+
 
   private volatile boolean testDone = false;
-  
+
   private final List<ZkController> zkControllers = Collections.synchronizedList(new ArrayList<>());
   private final List<Overseer> overseers = Collections.synchronizedList(new ArrayList<>());
   private final List<ZkStateReader> readers = Collections.synchronizedList(new ArrayList<>());
@@ -128,15 +129,15 @@ public class OverseerTest extends SolrTestCaseJ4 {
   private final List<CloudSolrClient> solrClients = Collections.synchronizedList(new ArrayList<>());
 
   private static final String COLLECTION = SolrTestCaseJ4.DEFAULT_TEST_COLLECTION_NAME;
-  
+
   public static class MockZKController{
-    
+
     private final SolrZkClient zkClient;
     private final ZkStateReader zkStateReader;
     private final String nodeName;
     private final Map<String, ElectionContext> electionContext = Collections.synchronizedMap(new HashMap<String, ElectionContext>());
     private List<Overseer> overseers;
-    
+
     public MockZKController(String zkAddress, String nodeName, List<Overseer> overseers) throws InterruptedException, TimeoutException, IOException, KeeperException {
       this.overseers = overseers;
       this.nodeName = nodeName;
@@ -146,14 +147,14 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
       zkStateReader = new ZkStateReader(zkClient);
       zkStateReader.createClusterStateWatchersAndUpdate();
-      
+
       // live node
       final String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
       zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
     }
 
     private void deleteNode(final String path) {
-      
+
       try {
         zkClient.delete(path, -1, true);
       } catch (NoNodeException e) {
@@ -257,7 +258,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       DocCollection dc = zkStateReader.getClusterState().getCollectionOrNull(collection);
       return getShardId(dc, coreNodeName);
     }
-    
+
     private String getShardId(DocCollection collection, String coreNodeName) {
       if (collection == null) return null;
       Map<String,Slice> slices = collection.getSlicesMap();
@@ -278,50 +279,50 @@ public class OverseerTest extends SolrTestCaseJ4 {
     public ZkStateReader getZkReader() {
       return zkStateReader;
     }
-  }    
-  
+  }
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     assumeWorkingMockito();
-    
+
     System.setProperty("solr.zkclienttimeout", "30000");
-    
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+
+    Path zkDir = createTempDir("zkData");
 
     server = new ZkTestServer(zkDir);
     server.run();
-    
+
     zkClient = server.getZkClient();
-    
+
     initCore();
   }
-  
-  
+
+
   @Before
   public void setUp() throws Exception {
     testDone = false;
     super.setUp();
   }
-  
+
   @AfterClass
   public static void afterClass() throws Exception {
     if (null != zkClient) {
       zkClient.printLayoutToStdOut();
     }
-    
+
     System.clearProperty("solr.zkclienttimeout");
-    
+
     if (null != server) {
       server.shutdown();
     }
   }
-  
+
   @After
   public void tearDown() throws Exception {
     testDone = true;
-    
+
     ExecutorService customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
-  
+
     for (ZkController zkController : zkControllers) {
       customThreadPool.submit( () -> zkController.close());
     }
@@ -333,7 +334,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     for (UpdateShardHandler updateShardHandler : updateShardHandlers) {
       customThreadPool.submit( () -> updateShardHandler.close());
     }
-    
+
     for (SolrClient solrClient : solrClients) {
       customThreadPool.submit( () -> IOUtils.closeQuietly(solrClient));
     }
@@ -341,22 +342,22 @@ public class OverseerTest extends SolrTestCaseJ4 {
     for (ZkStateReader reader : readers) {
       customThreadPool.submit( () -> reader.close());
     }
-    
+
     for (SolrZkClient solrZkClient : zkClients) {
       customThreadPool.submit( () -> IOUtils.closeQuietly(solrZkClient));
     }
-    
+
     ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
-    
+
     customThreadPool = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory("closeThreadPool"));
 
-    
+
     for (Overseer overseer : overseers) {
       customThreadPool.submit( () -> overseer.close());
     }
 
     ExecutorUtil.shutdownAndAwaitTermination(customThreadPool);
-    
+
     overseers.clear();
     zkControllers.clear();
     httpShardHandlerFactorys.clear();
@@ -364,10 +365,10 @@ public class OverseerTest extends SolrTestCaseJ4 {
     solrClients.clear();
     readers.clear();
     zkClients.clear();
-    
+
     server.tryCleanSolrZkNode();
     server.makeSolrZkNode();
-    
+
     super.tearDown();
   }
 
@@ -379,7 +380,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
     try {
 
- 
+
       ZkController.createClusterZkNodes(zkClient);
 
       overseerClient = electNewOverseer(server.getZkAddress());
@@ -398,13 +399,13 @@ public class OverseerTest extends SolrTestCaseJ4 {
             "createNodeSet", "");
         ZkDistributedQueue q = overseers.get(0).getStateUpdateQueue();
         q.offer(Utils.toJSON(m));
-        
+
         for (int i = 0; i < numShards; i++) {
           assertNotNull("shard got no id?", mockController.publishState(COLLECTION, "core" + (i + 1), "node" + (i + 1), "shard" + ((i % 3) + 1), Replica.State.ACTIVE, 3, true, overseers.get(0)));
         }
-        
+
         reader.waitForState(COLLECTION, 30, TimeUnit.SECONDS, MiniSolrCloudCluster.expectedShardsAndActiveReplicas(3, 6));
-        
+
         final Map<String, Replica> rmap = reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap();
         assertEquals(rmap.toString(), 2, rmap.size());
         assertEquals(rmap.toString(), 2, reader.getClusterState().getCollection(COLLECTION).getSlice("shard2").getReplicasMap().size());
@@ -445,7 +446,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
           assertNotNull("shard got no id?", mockController.publishState(COLLECTION, "core" + (i + 1),
               "node" + (i + 1), "shard" + ((i % 3) + 1), Replica.State.ACTIVE, 3, true, overseers.get(0)));
         }
-        
+
         reader.waitForState(COLLECTION, 30, TimeUnit.SECONDS, MiniSolrCloudCluster.expectedShardsAndActiveReplicas(3, 3));
 
         assertEquals(1, reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap().size());
@@ -468,7 +469,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
           assertNotNull("shard got no id?", mockController.publishState("collection2",
               "core" + (i + 1), "node" + (i + 1), "shard" + ((i % 3) + 1), Replica.State.ACTIVE, 3, true, overseers.get(0)));
         }
-        
+
         reader.waitForState("collection2", 30, TimeUnit.SECONDS, MiniSolrCloudCluster.expectedShardsAndActiveReplicas(3, 3));
 
         assertEquals(1, reader.getClusterState().getCollection("collection2").getSlice("shard1").getReplicasMap().size());
@@ -480,7 +481,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         assertNotNull(reader.getLeaderUrl("collection2", "shard2", 15000));
         assertNotNull(reader.getLeaderUrl("collection2", "shard3", 15000));
       }
-      
+
     } finally {
       if (mockController != null) {
         mockController.close();
@@ -541,7 +542,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
   private void waitForCollections(ZkStateReader stateReader, String... collections) throws InterruptedException, KeeperException, TimeoutException {
     int maxIterations = 100;
     while (0 < maxIterations--) {
-     
+
       final ClusterState state = stateReader.getClusterState();
       Set<String> availableCollections = state.getCollectionsMap().keySet();
       int availableCount = 0;
@@ -556,15 +557,15 @@ public class OverseerTest extends SolrTestCaseJ4 {
     }
     log.warn("Timeout waiting for collections: " + Arrays.asList(collections) + " state:" + stateReader.getClusterState());
   }
-  
+
   @Test
   public void testStateChange() throws Exception {
 
     ZkStateReader reader = null;
     SolrZkClient overseerClient = null;
-    
+
     try {
-      
+
       ZkController.createClusterZkNodes(zkClient);
 
       reader = new ZkStateReader(zkClient);
@@ -590,9 +591,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
           ZkStateReader.CORE_NODE_NAME_PROP, "core_node1",
           ZkStateReader.ROLES_PROP, "",
           ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
-      
+
       q.offer(Utils.toJSON(m));
-      
+
       waitForCollections(reader, COLLECTION);
       verifyReplicaStatus(reader, "collection1", "shard1", "core_node1", Replica.State.RECOVERING);
 
@@ -617,7 +618,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       close(reader);
     }
   }
-  
+
   private void verifyShardLeader(ZkStateReader reader, String collection, String shard, String expectedCore)
       throws InterruptedException, KeeperException, TimeoutException {
 
@@ -631,7 +632,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         (docCollection.getLeader(shard) != null) ? docCollection.getLeader(shard).getStr(ZkStateReader.CORE_NAME_PROP)
             : null);
   }
-  
+
   private Overseer getOpenOverseer() {
     return MiniSolrCloudCluster.getOpenOverseer(overseers);
   }
@@ -642,7 +643,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     SolrZkClient overseerClient = null;
     ZkStateReader reader = null;
     MockZKController mockController = null;
-    
+
     try {
 
       final String core = "core1";
@@ -651,47 +652,47 @@ public class OverseerTest extends SolrTestCaseJ4 {
       final int numShards = 1;
 
       ZkController.createClusterZkNodes(zkClient);
-      
+
       reader = new ZkStateReader(zkClient);
       reader.createClusterStateWatchersAndUpdate();
-      
+
       mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
-      
+
       overseerClient = electNewOverseer(server.getZkAddress());
-      
+
       mockController.createCollection(COLLECTION, 1);
-      
+
       ZkController zkController = createMockZkController(server.getZkAddress(), zkClient, reader);
-      
+
       mockController.publishState(COLLECTION, core, core_node, "shard1",
           Replica.State.RECOVERING, numShards, true, overseers.get(0));
-      
+
       waitForCollections(reader, COLLECTION);
       verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.RECOVERING);
-      
+
       int version = getClusterStateVersion(zkClient);
-      
+
       mockController.publishState(COLLECTION, core, core_node, "shard1", Replica.State.ACTIVE,
           numShards, true, overseers.get(0));
-      
+
       while (version == getClusterStateVersion(zkClient));
 
       verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.ACTIVE);
       version = getClusterStateVersion(zkClient);
-      
+
       mockController.publishState(COLLECTION, core, core_node, "shard1",
           Replica.State.RECOVERING, numShards, true, overseers.get(0));
-      
+
       overseerClient.close();
-      
+
       version = getClusterStateVersion(zkClient);
-      
+
       overseerClient = electNewOverseer(server.getZkAddress());
-      
+
       while (version == getClusterStateVersion(zkClient));
 
       verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.RECOVERING);
-      
+
       assertEquals("Live nodes count does not match", 1, reader
           .getClusterState().getLiveNodes().size());
       assertEquals(shard+" replica count does not match", 1, reader.getClusterState()
@@ -702,7 +703,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
       assertTrue(COLLECTION +" should remain after removal of the last core", // as of SOLR-5209 core removal does not cascade to remove the slice and collection
           reader.getClusterState().hasCollection(COLLECTION));
-      
+
       reader.waitForState(COLLECTION, 5000,
             TimeUnit.MILLISECONDS, (liveNodes, collectionState) -> collectionState != null && collectionState.getReplica(core_node) == null);
       assertTrue(core_node+" should be gone after publishing the null state",
@@ -734,7 +735,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         overseers.get(overseers.size() -1).getZkStateReader().getZkClient().close();
       }
       ZkController zkController = createMockZkController(server.getZkAddress(), zkClient, reader);
-      
+
       UpdateShardHandler updateShardHandler = new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT);
       updateShardHandlers.add(updateShardHandler);
       HttpShardHandlerFactory httpShardHandlerFactory = new HttpShardHandlerFactory();
@@ -769,7 +770,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       close(reader);
     }
   }
-  
+
   private AtomicInteger killCounter = new AtomicInteger();
 
   private class OverseerRestarter implements Runnable{
@@ -780,7 +781,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     public OverseerRestarter(String zkAddress) {
       this.zkAddress = zkAddress;
     }
-    
+
     @Override
     public void run() {
       try {
@@ -862,7 +863,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         }
         Thread.sleep(50);
       }
-      
+
       assertTrue(showQpeek(workQueue), workQueue.peek() == null);
       assertTrue(showQpeek(q),  q.peek() == null);
     } finally {
@@ -870,7 +871,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       close(reader);
     }
   }
-  
+
   private String showQpeek(ZkDistributedQueue q) throws KeeperException, InterruptedException {
     if (q == null) {
       return "";
@@ -879,7 +880,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     if (bytes == null) {
       return "";
     }
-    
+
     ZkNodeProps json = ZkNodeProps.load(bytes);
     return json.toString();
   }
@@ -940,9 +941,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
           mockController2.close();
           mockController2 = null;
         }
-        
+
         Thread.sleep(100);
-        
+
         timeout = new TimeOut(1, TimeUnit.SECONDS, TimeSource.NANO_TIME);
         while (!timeout.hasTimedOut()) {
           try {
@@ -980,7 +981,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
             e.printStackTrace();
           }
         }
-        
+
 
         mockController.close();
         mockController = null;
@@ -1026,26 +1027,26 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testDoubleAssignment() throws Exception {
-    
+
     SolrZkClient overseerClient = null;
     ZkStateReader reader = null;
     MockZKController mockController = null;
-    
+
     try {
-      
+
       ZkController.createClusterZkNodes(zkClient);
-      
+
       reader = new ZkStateReader(zkClient);
       reader.createClusterStateWatchersAndUpdate();
 
       mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
-      
+
       overseerClient = electNewOverseer(server.getZkAddress());
 
       mockController.createCollection(COLLECTION, 1);
-      
+
       ZkController zkController = createMockZkController(server.getZkAddress(), zkClient, reader);
-      
+
       mockController.publishState(COLLECTION, "core1", "core_node1", "shard1", Replica.State.RECOVERING, 1, true, overseers.get(0));
 
       waitForCollections(reader, COLLECTION);
@@ -1055,11 +1056,11 @@ public class OverseerTest extends SolrTestCaseJ4 {
       mockController.close();
 
       int version = getClusterStateVersion(zkClient);
-      
+
       mockController = new MockZKController(server.getZkAddress(), "node1", overseers);
 
       mockController.publishState(COLLECTION, "core1", "core_node1","shard1", Replica.State.RECOVERING, 1, true, overseers.get(0));
-      
+
       try {
         reader.waitForState(COLLECTION, 5, TimeUnit.SECONDS, (liveNodes, collectionState) -> version == zkController
             .getZkStateReader().getClusterState().getZkClusterStateVersion());
@@ -1067,7 +1068,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
         // okay
       }
       ClusterState state = reader.getClusterState();
-      
+
       int numFound = 0;
       Map<String, DocCollection> collectionsMap = state.getCollectionsMap();
       for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
@@ -1209,13 +1210,13 @@ public class OverseerTest extends SolrTestCaseJ4 {
     }
   }
 
-  
+
   @Test
   public void testReplay() throws Exception{
 
     SolrZkClient overseerClient = null;
     ZkStateReader reader = null;
-    
+
     try {
 
       ZkController.createClusterZkNodes(zkClient);
@@ -1249,9 +1250,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
           ZkStateReader.ROLES_PROP, "",
           ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
       queue.offer(Utils.toJSON(m));
-      
+
       overseerClient = electNewOverseer(server.getZkAddress());
-      
+
       //submit to proper queue
       queue = overseers.get(0).getStateUpdateQueue();
       m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(),
@@ -1263,11 +1264,11 @@ public class OverseerTest extends SolrTestCaseJ4 {
           ZkStateReader.ROLES_PROP, "",
           ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString());
       queue.offer(Utils.toJSON(m));
-      
+
       reader.waitForState(COLLECTION, 1000, TimeUnit.MILLISECONDS,
           (liveNodes, collectionState) -> collectionState != null && collectionState.getSlice("shard1") != null
               && collectionState.getSlice("shard1").getReplicas().size() == 3);
-      
+
       assertNotNull(reader.getClusterState().getCollection(COLLECTION).getSlice("shard1"));
       assertEquals(3, reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap().size());
     } finally {
@@ -1395,7 +1396,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       client.close();
     }
   }
-  
+
   private int getClusterStateVersion(SolrZkClient controllerClient)
       throws KeeperException, InterruptedException {
     return controllerClient.exists(ZkStateReader.CLUSTER_STATE, null, false).getVersion();
@@ -1433,7 +1434,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
 
   private ZkController createMockZkController(String zkAddress, SolrZkClient zkClient, ZkStateReader reader) throws InterruptedException, NoSuchFieldException, SecurityException, SessionExpiredException {
     ZkController zkController = mock(ZkController.class);
-    
+
     if (zkClient == null) {
       SolrZkClient newZkClient = new SolrZkClient(server.getZkAddress(), AbstractZkTestCase.TIMEOUT);
       Mockito.doAnswer(
@@ -1446,7 +1447,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     } else {
       doNothing().when(zkController).close();
     }
-    
+
     CoreContainer mockAlwaysUpCoreContainer = mock(CoreContainer.class,
         Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS));
     when(mockAlwaysUpCoreContainer.isShutDown()).thenReturn(testDone);  // Allow retry on session expiry
@@ -1458,7 +1459,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     when(zkController.getZkStateReader()).thenReturn(reader);
 
     when(zkController.getLeaderProps(anyString(), anyString(), anyInt())).thenCallRealMethod();
-    when(zkController.getLeaderProps(anyString(), anyString(), anyInt(), anyBoolean())).thenCallRealMethod();  
+    when(zkController.getLeaderProps(anyString(), anyString(), anyInt(), anyBoolean())).thenCallRealMethod();
     doReturn(getCloudDataProvider(zkAddress, zkClient, reader))
         .when(zkController).getSolrCloudManager();
     return zkController;
@@ -1562,7 +1563,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
               ZkStateReader.CORE_NODE_NAME_PROP, "core_node"+N);
 
           q.offer(Utils.toJSON(m));
-           
+
           {
             String shard = "shard"+ss;
             zkStateReader.waitForState(COLLECTION, 15000, TimeUnit.MILLISECONDS, (liveNodes, collectionState) -> collectionState != null && (collectionState.getSlice(shard) == null || collectionState.getSlice(shard).getReplicasMap().get("core_node"+N) == null));
@@ -1585,7 +1586,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       close(zkStateReader);
     }
   }
-  
+
   @Test
   public void testLatchWatcher() throws InterruptedException {
     OverseerTaskQueue.LatchWatcher latch1 = new OverseerTaskQueue.LatchWatcher();
@@ -1599,7 +1600,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
     latch1.await(10000);// Expecting no wait
     after = System.nanoTime();
     assertTrue(TimeUnit.NANOSECONDS.toMillis(after-before) < 1000);
-    
+
     final AtomicBoolean expectedEventProcessed = new AtomicBoolean(false);
     final AtomicBoolean doneWaiting = new AtomicBoolean(false);
     final OverseerTaskQueue.LatchWatcher latch2 = new OverseerTaskQueue.LatchWatcher(Event.EventType.NodeCreated);
diff --git a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
index b362bfa..4e3d62e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SaslZkACLProviderTest.java
@@ -20,6 +20,7 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
+import java.nio.file.Path;
 
 import org.apache.lucene.util.Constants;
 import org.apache.solr.SolrTestCaseJ4;
@@ -30,7 +31,6 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkACLProvider;
 import org.apache.solr.util.BadZookeeperThreadsFilter;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -72,10 +72,9 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
     log.info("####SETUP_START " + getTestName());
     createTempDir();
 
-    String zkDir = createTempDir() + File.separator
-        + "zookeeper/server1/data";
+    Path zkDir = createTempDir().resolve("zookeeper/server1/data");
     log.info("ZooKeeper dataDir:" + zkDir);
-    zkServer = new SaslZkTestServer(zkDir, createTempDir() + File.separator + "miniKdc");
+    zkServer = new SaslZkTestServer(zkDir, createTempDir().resolve("miniKdc"));
     zkServer.run();
 
     System.setProperty("zkHost", zkServer.getZkAddress());
@@ -175,19 +174,14 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
    * A ZkTestServer with Sasl support
    */
   public static class SaslZkTestServer extends ZkTestServer {
-    private String kdcDir;
+    private Path kdcDir;
     private KerberosTestServices kerberosTestServices;
 
-    public SaslZkTestServer(String zkDir, String kdcDir) throws Exception {
+    public SaslZkTestServer(Path zkDir, Path kdcDir) throws Exception {
       super(zkDir);
       this.kdcDir = kdcDir;
     }
 
-    public SaslZkTestServer(String zkDir, int port, String kdcDir) throws KeeperException, InterruptedException {
-      super(zkDir, port);
-      this.kdcDir = kdcDir;
-    }
-
     @Override
     public void run() throws InterruptedException, IOException {
       try {
@@ -195,12 +189,12 @@ public class SaslZkACLProviderTest extends SolrTestCaseJ4 {
         // can match "solr" rather than "solr/host@DOMAIN"
         System.setProperty("zookeeper.kerberos.removeRealmFromPrincipal", "true");
         System.setProperty("zookeeper.kerberos.removeHostFromPrincipal", "true");
-        File keytabFile = new File(kdcDir, "keytabs");
+        File keytabFile = kdcDir.resolve("keytabs").toFile();
         String zkClientPrincipal = "solr";
         String zkServerPrincipal = "zookeeper/localhost";
 
         kerberosTestServices = KerberosTestServices.builder()
-            .withKdc(new File(kdcDir))
+            .withKdc(kdcDir.toFile())
             .withJaasConfiguration(zkClientPrincipal, keytabFile, zkServerPrincipal, keytabFile)
            
             .build();
diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
index e6fc954..e8d0e92 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java
@@ -46,7 +46,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
 
   protected ZkTestServer zkServer;
 
-  protected String zkDir;
+  protected Path zkDir;
 
   private SolrZkClient zkClient;
 
@@ -67,7 +67,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
 
     System.setProperty("zkClientTimeout", "8000");
 
-    zkDir = tmpDir.resolve("zookeeper" + System.nanoTime()).resolve("server1").resolve("data").toString();
+    zkDir = tmpDir.resolve("zookeeper" + System.nanoTime()).resolve("server1").resolve("data");
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
     System.setProperty("zkHost", zkServer.getZkAddress());
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
index 41c5463..e7f3537 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
@@ -22,10 +22,9 @@ import java.io.PrintWriter;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
 import java.util.Collection;
-import java.util.HashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -78,7 +77,7 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
   public void setUp() throws Exception {
     super.setUp();
     final Path testDir = createTempDir();
-    String zkDir = testDir.resolve("zookeeper/server1/data").toString();
+    final Path zkDir = testDir.resolve("zookeeper/server1/data");
     zkTestServer = new ZkTestServer(zkDir);
     zkTestServer.run();
     zkTestServer.setZKDatabase(new FailureDuringCopyZKDatabase(zkTestServer.getZKDatabase(), zkTestServer));
@@ -204,7 +203,7 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
     }
 
     @Override
-    public synchronized LinkedList<Proposal> getCommittedLog() {
+    public synchronized List<Proposal> getCommittedLog() {
       return zkdb.getCommittedLog();
     }
 
@@ -214,11 +213,6 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
     }
 
     @Override
-    public void setDataTreeInit(boolean b) {
-      zkdb.setDataTreeInit(b);
-    }
-
-    @Override
     public Collection<Long> getSessions() {
       return zkdb.getSessions();
     }
@@ -259,7 +253,7 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
     }
 
     @Override
-    public HashSet<String> getEphemerals(long sessionId) {
+    public Set<String> getEphemerals(long sessionId) {
       return zkdb.getEphemerals(sessionId);
     }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDistributedMap.java b/solr/core/src/test/org/apache/solr/cloud/TestDistributedMap.java
index 417cf2f..ba380d1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDistributedMap.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDistributedMap.java
@@ -37,7 +37,7 @@ public class TestDistributedMap extends SolrTestCaseJ4 {
   @BeforeClass
   public static void setUpClass() throws Exception {
     zkDir = createTempDir("TestDistributedMap");
-    zkServer = new ZkTestServer(zkDir.toFile().getAbsolutePath());
+    zkServer = new ZkTestServer(zkDir);
     zkServer.run();
   }
   
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java
index 8e6057d..6bbfe1c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java
@@ -42,7 +42,7 @@ public class TestLeaderElectionZkExpiry extends SolrTestCaseJ4 {
 
   @Test
   public void testLeaderElectionWithZkExpiry() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     Path ccDir = createTempDir("testLeaderElectionWithZkExpiry-solr");
     CoreContainer cc = createCoreContainer(ccDir, SOLRXML);
     final ZkTestServer server = new ZkTestServer(zkDir);
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java b/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
index a89d3e3..9ab74ec 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestZkChroot.java
@@ -34,14 +34,14 @@ public class TestZkChroot extends SolrTestCaseJ4 {
   private Path home;
   
   protected ZkTestServer zkServer;
-  protected String zkDir;
+  protected Path zkDir;
   
   @Override
   @Before
   public void setUp() throws Exception {
     super.setUp();
 
-    zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    zkDir = createTempDir("zkData");
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
     home = Paths.get(SolrJettyTestBase.legacyExampleCollection1SolrHome());
diff --git a/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
index 36fb989..41413a3 100644
--- a/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/VMParamsZkACLAndCredentialsProvidersTest.java
@@ -16,9 +16,9 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
+import java.nio.file.Path;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
@@ -41,7 +41,7 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
   
   protected ZkTestServer zkServer;
   
-  protected String zkDir;
+  protected Path zkDir;
   
   @BeforeClass
   public static void beforeClass() {
@@ -59,8 +59,7 @@ public class VMParamsZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 {
     log.info("####SETUP_START " + getTestName());
     createTempDir();
     
-    zkDir = createTempDir() + File.separator
-        + "zookeeper/server1/data";
+    zkDir = createTempDir().resolve("zookeeper/server1/data");
     log.info("ZooKeeper dataDir:" + zkDir);
     zkServer = new ZkTestServer(zkDir);
     zkServer.run(false);
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
index 4363382..94ecf39 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.io.PrintStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.List;
 
@@ -55,25 +56,25 @@ public class ZkCLITest extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   protected ZkTestServer zkServer;
-  
-  protected String zkDir;
-  
+
+  protected Path zkDir;
+
   private String solrHome;
 
   private SolrZkClient zkClient;
 
   protected static final String SOLR_HOME = SolrTestCaseJ4.TEST_HOME();
-  
+
   @BeforeClass
   public static void beforeClass() {
     System.setProperty("solrcloud.skip.autorecovery", "true");
   }
-  
+
   @AfterClass
   public static void afterClass() throws InterruptedException {
     System.clearProperty("solrcloud.skip.autorecovery");
   }
-  
+
   @Override
   public void setUp() throws Exception {
     super.setUp();
@@ -81,11 +82,10 @@ public class ZkCLITest extends SolrTestCaseJ4 {
 
     String exampleHome = SolrJettyTestBase.legacyExampleCollection1SolrHome();
 
-    File tmpDir = createTempDir().toFile();
+    Path tmpDir = createTempDir();
     solrHome = exampleHome;
 
-    zkDir = tmpDir.getAbsolutePath() + File.separator
-        + "zookeeper/server1/data";
+    zkDir = tmpDir.resolve("zookeeper/server1/data");
     log.info("ZooKeeper dataDir:" + zkDir);
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
@@ -94,13 +94,13 @@ public class ZkCLITest extends SolrTestCaseJ4 {
     zkClient.makePath("/solr", false, true);
     zkClient.close();
 
-    
+
     this.zkClient = new SolrZkClient(zkServer.getZkAddress(),
         AbstractZkTestCase.TIMEOUT);
-    
+
     log.info("####SETUP_END " + getTestName());
   }
-  
+
   @Test
   public void testCmdConstants() throws Exception {
     assertEquals("upconfig", ZkCLI.UPCONFIG);
@@ -113,12 +113,12 @@ public class ZkCLITest extends SolrTestCaseJ4 {
   public void testBootstrapWithChroot() throws Exception {
     String chroot = "/foo/bar";
     assertFalse(zkClient.exists(chroot, true));
-    
+
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress() + chroot,
         "-cmd", "bootstrap", "-solrhome", this.solrHome};
-    
+
     ZkCLI.main(args);
-    
+
     assertTrue(zkClient.exists(chroot + ZkConfigManager.CONFIGS_ZKNODE
         + "/collection1", true));
   }
@@ -211,7 +211,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
     ZkCLI.setStdout(myOut);
 
     ZkCLI.main(args);
-    
+
     final String standardOutput = byteStream.toString(StandardCharsets.UTF_8.name());
     String separator = System.lineSeparator();
     assertEquals("/test (1)" + separator + " /test/path (0)" + separator + separator, standardOutput);
@@ -220,7 +220,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
   @Test
   public void testUpConfigLinkConfigClearZk() throws Exception {
     File tmpDir = createTempDir().toFile();
-    
+
     // test upconfig
     String confsetname = "confsetone";
     final String[] upconfigArgs;
@@ -240,21 +240,21 @@ public class ZkCLITest extends SolrTestCaseJ4 {
           "-confname", confsetname};
     }
     ZkCLI.main(upconfigArgs);
-    
+
     assertTrue(zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + confsetname, true));
 
     // print help
     // ZkCLI.main(new String[0]);
-    
+
     // test linkconfig
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
         "linkconfig", "-collection", "collection1", "-confname", confsetname};
     ZkCLI.main(args);
-    
+
     ZkNodeProps collectionProps = ZkNodeProps.load(zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/collection1", null, null, true));
     assertTrue(collectionProps.containsKey("configName"));
     assertEquals(confsetname, collectionProps.getStr("configName"));
-    
+
     // test down config
     File confDir = new File(tmpDir,
         "solrtest-confdropspot-" + this.getClass().getName() + "-" + System.nanoTime());
@@ -263,11 +263,11 @@ public class ZkCLITest extends SolrTestCaseJ4 {
     args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
         "downconfig", "-confdir", confDir.getAbsolutePath(), "-confname", confsetname};
     ZkCLI.main(args);
-    
+
     File[] files = confDir.listFiles();
     List<String> zkFiles = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE + "/" + confsetname, null, true);
     assertEquals(files.length, zkFiles.size());
-    
+
     File sourceConfDir = new File(ExternalPaths.TECHPRODUCTS_CONFIGSET);
     // filter out all directories starting with . (e.g. .svn)
     Collection<File> sourceFiles = FileUtils.listFiles(sourceConfDir, TrueFileFilter.INSTANCE, new RegexFileFilter("[^\\.].*"));
@@ -282,8 +282,8 @@ public class ZkCLITest extends SolrTestCaseJ4 {
           assertTrue(relativePathofFile+" content changed",FileUtils.contentEquals(sourceFile,downloadedFile));
         }
     }
-    
-   
+
+
     // test reset zk
     args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
         "clear", "/"};
@@ -291,7 +291,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
 
     assertEquals(0, zkClient.getChildren("/", null, true).size());
   }
-  
+
   @Test
   public void testGet() throws Exception {
     String getNode = "/getNode";
@@ -305,7 +305,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
   @Test
   public void testGetFile() throws Exception {
     File tmpDir = createTempDir().toFile();
-    
+
     String getNode = "/getFileNode";
     byte [] data = "getFileNode-data".getBytes(StandardCharsets.UTF_8);
     this.zkClient.create(getNode, data, CreateMode.PERSISTENT, true);
@@ -331,9 +331,10 @@ public class ZkCLITest extends SolrTestCaseJ4 {
     assertEquals(e.code(), KeeperException.Code.NONODE);
   }
 
-  @Test(expected = SolrException.class)
-  public void testInvalidZKAddress() throws SolrException{
-    SolrZkClient zkClient = new SolrZkClient("----------:33332", 100);
+  public void testInvalidZKAddress() throws SolrException {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      new SolrZkClient("----------:33332", 100);
+    });
     zkClient.close();
   }
 
@@ -341,19 +342,19 @@ public class ZkCLITest extends SolrTestCaseJ4 {
   public void testSetClusterProperty() throws Exception {
     ClusterProperties properties = new ClusterProperties(zkClient);
     // add property urlScheme=http
-    String[] args = new String[] {"-zkhost", zkServer.getZkAddress(),
+    String[] args = new String[]{"-zkhost", zkServer.getZkAddress(),
         "-cmd", "CLUSTERPROP", "-name", "urlScheme", "-val", "http"};
     ZkCLI.main(args);
     assertEquals("http", properties.getClusterProperty("urlScheme", "none"));
 
     // remove it again
-    args = new String[] {"-zkhost", zkServer.getZkAddress(),
+    args = new String[]{"-zkhost", zkServer.getZkAddress(),
         "-cmd", "CLUSTERPROP", "-name", "urlScheme"};
     ZkCLI.main(args);
     assertNull(properties.getClusterProperty("urlScheme", (String) null));
 
   }
-  
+
   @Test
   public void testUpdateAcls() throws Exception {
     try {
@@ -361,7 +362,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
       System.setProperty(VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME, "user");
       System.setProperty(VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME, "pass");
 
-      String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd", "updateacls", "/"};
+      String[] args = new String[]{"-zkhost", zkServer.getZkAddress(), "-cmd", "updateacls", "/"};
       ZkCLI.main(args);
     } finally {
       // Need to clear these before we open the next SolrZkClient
@@ -369,7 +370,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
       System.clearProperty(VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_USERNAME_VM_PARAM_NAME);
       System.clearProperty(VMParamsAllAndReadonlyDigestZkACLProvider.DEFAULT_DIGEST_READONLY_PASSWORD_VM_PARAM_NAME);
     }
-    
+
     boolean excepted = false;
     try (SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), AbstractDistribZkTestBase.DEFAULT_CONNECTION_TIMEOUT)) {
       zkClient.getData("/", null, null, true);
@@ -385,7 +386,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
     zkServer.shutdown();
     super.tearDown();
   }
-  
+
   private void printLayout(String zkHost) throws Exception {
     SolrZkClient zkClient = new SolrZkClient(zkHost, AbstractZkTestCase.TIMEOUT);
     zkClient.printLayoutToStdOut();
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
index 45c4812..df478bc 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -87,7 +88,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
                  ZkController.generateNodeName("foo-bar", "77", "/solr/sub_dir/"));
 
     // setup a SolrZkClient to do some getBaseUrlForNodeName testing
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
 
     ZkTestServer server = new ZkTestServer(zkDir);
     try {
@@ -166,7 +167,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
 
   @Test
   public void testReadConfigName() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     CoreContainer cc = null;
 
     ZkTestServer server = new ZkTestServer(zkDir);
@@ -215,7 +216,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
   }
 
   public void testGetHostName() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     CoreContainer cc = null;
 
     ZkTestServer server = new ZkTestServer(zkDir);
@@ -264,7 +265,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
 
     assumeWorkingMockito();
     final String collectionName = "testPublishAndWaitForDownStates";
-    String zkDir = createTempDir(collectionName).toFile().getAbsolutePath();
+    Path zkDir = createTempDir(collectionName);
     CoreContainer cc = null;
 
     String nodeName = "127.0.0.1:8983_solr";
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
index 276a04c..6a09162 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import java.nio.file.Path;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
@@ -50,7 +51,7 @@ public class ZkSolrClientTest extends SolrTestCaseJ4 {
     }
 
     ZkConnection(boolean makeRoot) throws Exception {
-      String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+      Path zkDir = createTempDir("zkData");
       server = new ZkTestServer(zkDir);
       server.run();
 
@@ -105,7 +106,7 @@ public class ZkSolrClientTest extends SolrTestCaseJ4 {
   }
 
   public void testReconnect() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = null;
     server = new ZkTestServer(zkDir);
     server.run();
@@ -198,7 +199,7 @@ public class ZkSolrClientTest extends SolrTestCaseJ4 {
   }
   
   public void testZkCmdExectutor() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = null;
 
     try {
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/AssignTest.java
index 512a397..c6971de 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/AssignTest.java
@@ -23,6 +23,7 @@ import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -92,7 +93,7 @@ public class AssignTest extends SolrTestCaseJ4 {
 
   @Test
   public void testIdIsUnique() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = new ZkTestServer(zkDir);
     Object fixedValue = new Object();
     String[] collections = new String[]{"c1","c2","c3","c4","c5","c6","c7","c8","c9"};
@@ -139,7 +140,7 @@ public class AssignTest extends SolrTestCaseJ4 {
 
   @Test
   public void testBuildCoreName() throws Exception {
-    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("zkData");
     ZkTestServer server = new ZkTestServer(zkDir);
     server.run();
     try (SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), 10000)) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistribStateManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistribStateManager.java
index 731d6e8..3ed6088 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistribStateManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimDistribStateManager.java
@@ -62,7 +62,7 @@ public class TestSimDistribStateManager extends SolrTestCaseJ4 {
     if (simulated) {
       root = SimDistribStateManager.createNewRootNode();
     } else {
-      zkTestServer = new ZkTestServer(createTempDir("zkDir").toString());
+      zkTestServer = new ZkTestServer(createTempDir("zkDir"));
       zkTestServer.run();
     }
     reInit();
diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
index 581deec..f4c5bb2 100644
--- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateReaderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud.overseer;
 
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -61,7 +62,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
   }
 
   public void testStateFormatUpdate(boolean explicitRefresh, boolean isInteresting) throws Exception {
-    String zkDir = createTempDir("testStateFormatUpdate").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testStateFormatUpdate");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -137,7 +138,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
   }
 
   public void testExternalCollectionWatchedNotWatched() throws Exception{
-    String zkDir = createTempDir("testExternalCollectionWatchedNotWatched").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testExternalCollectionWatchedNotWatched");
     ZkTestServer server = new ZkTestServer(zkDir);
     SolrZkClient zkClient = null;
     ZkStateReader reader = null;
@@ -175,7 +176,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
   }
 
   public void testCollectionStateWatcherCaching() throws Exception  {
-    String zkDir = createTempDir("testCollectionStateWatcherCaching").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testCollectionStateWatcherCaching");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -223,7 +224,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
   }
 
   public void testWatchedCollectionCreation() throws Exception {
-    String zkDir = createTempDir("testWatchedCollectionCreation").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testWatchedCollectionCreation");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
index a47aa7d..bc2b42d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/overseer/ZkStateWriterTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr.cloud.overseer;
 
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -60,7 +61,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
   }
 
   public void testZkStateWriterBatching() throws Exception {
-    String zkDir = createTempDir("testZkStateWriterBatching").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testZkStateWriterBatching");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -110,7 +111,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
   }
 
   public void testSingleLegacyCollection() throws Exception {
-    String zkDir = createTempDir("testSingleLegacyCollection").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testSingleLegacyCollection");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -150,7 +151,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
   }
 
   public void testSingleExternalCollection() throws Exception {
-    String zkDir = createTempDir("testSingleExternalCollection").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testSingleExternalCollection");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -192,7 +193,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
   }
 
   public void testExternalModificationToSharedClusterState() throws Exception {
-    String zkDir = createTempDir("testExternalModification").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testExternalModification");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
@@ -266,7 +267,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
   }
 
   public void testExternalModificationToStateFormat2() throws Exception {
-    String zkDir = createTempDir("testExternalModificationToStateFormat2").toFile().getAbsolutePath();
+    Path zkDir = createTempDir("testExternalModificationToStateFormat2");
 
     ZkTestServer server = new ZkTestServer(zkDir);
 
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
index 3e9dec6..4f29d8a 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/ZookeeperStatusHandlerTest.java
@@ -18,6 +18,7 @@
 package org.apache.solr.handler.admin;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.util.List;
 import java.util.Map;
@@ -37,8 +38,12 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(1)
@@ -63,7 +68,6 @@ public class ZookeeperStatusHandlerTest extends SolrCloudTestCase {
     NOTE: We do not currently test with multiple zookeepers, but the only difference is that there are multiple "details" objects and mode is "ensemble"... 
    */
   @Test
-  // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 6-Sep-2018
   public void monitorZookeeper() throws IOException, SolrServerException, InterruptedException, ExecutionException, TimeoutException {
     URL baseUrl = cluster.getJettySolrRunner(0).getBaseUrl();
     HttpSolrClient solr = new HttpSolrClient.Builder(baseUrl.toString()).build();
diff --git a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
index 2560116..0899077 100644
--- a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
+++ b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaThreadSafety.java
@@ -90,7 +90,7 @@ public class TestManagedSchemaThreadSafety extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void startZkServer() throws Exception {
-    zkServer = new ZkTestServer(createTempDir().toString());
+    zkServer = new ZkTestServer(createTempDir());
     zkServer.run();
     loaderPath = createTempDir();
   }
diff --git a/solr/licenses/zookeeper-3.4.14.jar.sha1 b/solr/licenses/zookeeper-3.4.14.jar.sha1
deleted file mode 100644
index 6adbf71..0000000
--- a/solr/licenses/zookeeper-3.4.14.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-c114c1e1c8172a7cd3f6ae39209a635f7a06c1a1
diff --git a/solr/licenses/zookeeper-3.5.5.jar.sha1 b/solr/licenses/zookeeper-3.5.5.jar.sha1
new file mode 100644
index 0000000..56c9d99
--- /dev/null
+++ b/solr/licenses/zookeeper-3.5.5.jar.sha1
@@ -0,0 +1 @@
+dd9c924e9d4be7c79e46261691e96d030736a8ac
diff --git a/solr/licenses/zookeeper-jute-3.5.5.jar.sha1 b/solr/licenses/zookeeper-jute-3.5.5.jar.sha1
new file mode 100644
index 0000000..a076163
--- /dev/null
+++ b/solr/licenses/zookeeper-jute-3.5.5.jar.sha1
@@ -0,0 +1 @@
+3785011a665bd5c7dedd025110543d967f17f8e3
diff --git a/solr/licenses/zookeeper-jute-LICENSE-ASL.txt b/solr/licenses/zookeeper-jute-LICENSE-ASL.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/solr/licenses/zookeeper-jute-LICENSE-ASL.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/solr/licenses/zookeeper-jute-NOTICE.txt b/solr/licenses/zookeeper-jute-NOTICE.txt
new file mode 100644
index 0000000..54c971e
--- /dev/null
+++ b/solr/licenses/zookeeper-jute-NOTICE.txt
@@ -0,0 +1,5 @@
+Apache Zookeeper
+Copyright 2011 The Apache Software Foundation
+
+This product includes software developed by
+The Apache Software Foundation (http://www.apache.org/).
diff --git a/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc b/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
index b292e1a..224dd00 100644
--- a/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
+++ b/solr/solr-ref-guide/src/setting-up-an-external-zookeeper-ensemble.adoc
@@ -83,6 +83,7 @@ The file should have the following information to start:
 tickTime=2000
 dataDir=/var/lib/zookeeper
 clientPort=2181
+4lw.commands.whitelist=mntr,conf,ruok
 ----
 
 The parameters are as follows:
@@ -93,6 +94,8 @@ The parameters are as follows:
 
 `clientPort`:: This is the port on which Solr will access ZooKeeper.
 
+`4lw.commands.whitelist`:: This allows the Solr admin UI to query ZooKeeper. Optionally use "*" to enable all "4 letter words", the three listed will enable the admin UI.
+
 These are the basic parameters that need to be in use on each ZooKeeper node, so this file must be copied to or created on each node.
 
 Next we'll customize this configuration to work within an ensemble.
@@ -110,6 +113,7 @@ Once complete, your `zoo.cfg` file might look like this:
 tickTime=2000
 dataDir=/var/lib/zookeeper
 clientPort=2181
+4lw.commands.whitelist=mntr,conf,ruok
 
 initLimit=5
 syncLimit=2
@@ -153,6 +157,7 @@ On the second node, update `<ZOOKEEPER_HOME>/conf/zoo.cfg` file so it matches th
 tickTime=2000
 dataDir=/var/lib/zookeeper
 clientPort=2181
+4lw.commands.whitelist=mntr,conf,ruok
 
 initLimit=5
 syncLimit=2
@@ -176,6 +181,7 @@ On the third node, update `<ZOOKEEPER_HOME>/conf/zoo.cfg` file so it matches the
 tickTime=2000
 dataDir=/var/lib/zookeeper
 clientPort=2181
+4lw.commands.whitelist=mntr,conf,ruok
 
 initLimit=5
 syncLimit=2
diff --git a/solr/solrj/ivy.xml b/solr/solrj/ivy.xml
index c158510..bdaef62 100644
--- a/solr/solrj/ivy.xml
+++ b/solr/solrj/ivy.xml
@@ -29,6 +29,7 @@
 
   <dependencies>
     <dependency org="org.apache.zookeeper" name="zookeeper" rev="${/org.apache.zookeeper/zookeeper}" conf="compile"/>
+    <dependency org="org.apache.zookeeper" name="zookeeper-jute" rev="${/org.apache.zookeeper/zookeeper}" conf="compile"/>
     <dependency org="org.apache.httpcomponents" name="httpclient" rev="${/org.apache.httpcomponents/httpclient}" conf="compile"/>
     <dependency org="org.apache.httpcomponents" name="httpmime" rev="${/org.apache.httpcomponents/httpmime}" conf="compile"/>
     <dependency org="org.apache.httpcomponents" name="httpcore" rev="${/org.apache.httpcomponents/httpcore}" conf="compile"/>
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 5f8d28d..0611ca5 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -197,6 +197,7 @@ public class SolrZkClient implements Closeable {
       } catch (InterruptedException e1) {
         Thread.currentThread().interrupt();
       }
+      zkConnManagerCallbackExecutor.shutdown();
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
     assert ObjectReleaseTracker.track(this);
diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
index 6f14f8f..420f9fb 100644
--- a/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/cloud/SolrZkClientTest.java
@@ -19,6 +19,7 @@ package org.apache.solr.common.cloud;
 import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
 import java.security.NoSuchAlgorithmException;
 import java.util.Collection;
 import java.util.Collections;
@@ -67,7 +68,7 @@ public class SolrZkClientTest extends SolrCloudTestCase {
     final String SCHEME = "digest";
     final String AUTH = "user:pass";
 
-    String zkDir = createTempDir().toString();
+    Path zkDir = createTempDir();
     log.info("ZooKeeper dataDir:" + zkDir);
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/TestZkConfigManager.java b/solr/solrj/src/test/org/apache/solr/common/cloud/TestZkConfigManager.java
index d656851..961d7a9 100644
--- a/solr/solrj/src/test/org/apache/solr/common/cloud/TestZkConfigManager.java
+++ b/solr/solrj/src/test/org/apache/solr/common/cloud/TestZkConfigManager.java
@@ -43,7 +43,7 @@ public class TestZkConfigManager extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void startZkServer() throws Exception {
-    zkServer = new ZkTestServer(createTempDir("zkData").toString());
+    zkServer = new ZkTestServer(createTempDir("zkData"));
     zkServer.run();
   }
 
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index da8520e..66e06ef 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -168,6 +168,7 @@ import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.solr.cloud.SolrZkServer.ZK_WHITELIST_PROPERTY;
 import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
@@ -299,6 +300,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
     System.setProperty("solr.clustering.enabled", "false");
     System.setProperty("solr.peerSync.useRangeVersions", String.valueOf(random().nextBoolean()));
     System.setProperty("solr.cloud.wait-for-updates-with-stale-state-pause", "500");
+    System.setProperty(ZK_WHITELIST_PROPERTY, "*");
     startTrackingSearchers();
     ignoreException("ignore_exception");
     newRandomConfig();
@@ -350,6 +352,7 @@ public abstract class SolrTestCaseJ4 extends SolrTestCase {
       System.clearProperty("solr.peerSync.useRangeVersions");
       System.clearProperty("solr.cloud.wait-for-updates-with-stale-state-pause");
       System.clearProperty("solr.zkclienttmeout");
+      System.clearProperty(ZK_WHITELIST_PROPERTY);
       HttpClientUtil.resetHttpClientBuilder();
       Http2SolrClient.resetSslContextFactory();
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 444649d..5ba0ee2 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -46,7 +47,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTestCase {
-  
+
   private static final String REMOVE_VERSION_FIELD = "remove.version.field";
   private static final String ENABLE_UPDATE_LOG = "enable.update.log";
   private static final String ZK_HOST = "zkHost";
@@ -66,12 +67,11 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
   @Override
   public void distribSetUp() throws Exception {
     super.distribSetUp();
-    
-    String zkDir = testDir.getAbsolutePath() + File.separator
-    + "zookeeper/server1/data";
+
+    Path zkDir = testDir.toPath().resolve("zookeeper/server1/data");
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
-    
+
     System.setProperty(ZK_HOST, zkServer.getZkAddress());
     System.setProperty(ENABLE_UPDATE_LOG, "true");
     System.setProperty(REMOVE_VERSION_FIELD, "true");
@@ -86,15 +86,15 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
     System.setProperty("solr.test.sys.prop1", "propone");
     System.setProperty("solr.test.sys.prop2", "proptwo");
   }
-  
+
   protected String getCloudSolrConfig() {
     return "solrconfig-tlog.xml";
   }
-  
+
   protected String getCloudSchemaFile() {
     return getSchemaFile();
   }
-  
+
   @Override
   protected void createServers(int numShards) throws Exception {
     // give everyone there own solrhome
@@ -110,7 +110,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
         .createCollection("control_collection", 1, 1)
         .setCreateNodeSet(controlJetty.getNodeName())
         .process(controlClient).isSuccess());
-    
+
     ZkStateReader zkStateReader = jettys.get(0).getCoreContainer().getZkController()
         .getZkStateReader();
 
@@ -132,17 +132,17 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
     shards = sb.toString();
 
   }
-  
+
   protected void waitForRecoveriesToFinish(String collection, ZkStateReader zkStateReader, boolean verbose)
       throws Exception {
     waitForRecoveriesToFinish(collection, zkStateReader, verbose, true);
   }
-  
+
   protected void waitForRecoveriesToFinish(String collection, ZkStateReader zkStateReader, boolean verbose, boolean failOnTimeout)
       throws Exception {
     waitForRecoveriesToFinish(collection, zkStateReader, verbose, failOnTimeout, 330);
   }
-  
+
   public static void waitForRecoveriesToFinish(String collection,
       ZkStateReader zkStateReader, boolean verbose, boolean failOnTimeout, long timeoutSeconds)
       throws Exception {
@@ -211,7 +211,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
     });
     log.info("Collection has disappeared - collection: " + collection);
   }
-  
+
   static void waitForNewLeader(CloudSolrClient cloudClient, String shardName, Replica oldLeader, TimeOut timeOut)
       throws Exception {
     log.info("Will wait for a node to become leader for {} secs", timeOut.timeLeft(SECONDS));
@@ -235,11 +235,11 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
 
       Thread.sleep(100);
     }
-    
+
     zkStateReader.waitForState("collection1", timeOut.timeLeft(SECONDS), TimeUnit.SECONDS, (liveNodes, docCollection) -> {
       if (docCollection == null)
         return false;
-      
+
       Slice slice = docCollection.getSlice(shardName);
       if (slice != null && slice.getLeader() != null && !slice.getLeader().equals(oldLeader) && slice.getLeader().getState() == Replica.State.ACTIVE) {
         log.info("Old leader {}, new leader {}. New leader got elected in {} ms", oldLeader, slice.getLeader(), timeOut.timeElapsed(MILLISECONDS) );
@@ -256,7 +256,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
             && collectionState.getSlice(shard).getReplicasMap().get(coreNodeName) != null
             && collectionState.getSlice(shard).getReplicasMap().get(coreNodeName).getState() == expectedState);
   }
-  
+
   protected static void assertAllActive(String collection, ZkStateReader zkStateReader)
       throws KeeperException, InterruptedException {
 
@@ -266,7 +266,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
       if (docCollection == null || docCollection.getSlices() == null) {
         throw new IllegalArgumentException("Cannot find collection:" + collection);
       }
-      
+
       Map<String,Slice> slices = docCollection.getSlicesMap();
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
         Slice slice = entry.getValue();
@@ -282,7 +282,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
         }
       }
   }
-  
+
   @Override
   public void distribTearDown() throws Exception {
     resetExceptionIgnores();
@@ -309,7 +309,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
 
     }
   }
-  
+
   protected void printLayout() throws Exception {
     SolrZkClient zkClient = new SolrZkClient(zkServer.getZkHost(), AbstractZkTestCase.TIMEOUT);
     zkClient.printLayoutToStdOut();
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
index 47ef259..f27525e 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud;
 
 import java.io.File;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
 
 import org.apache.solr.SolrTestCaseJ4;
 import org.junit.AfterClass;
@@ -48,12 +49,12 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
 
   protected volatile static ZkTestServer zkServer;
 
-  protected volatile static String zkDir;
+  protected volatile static Path zkDir;
 
 
   @BeforeClass
   public static void azt_beforeClass() throws Exception {
-    zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+    zkDir = createTempDir("zkData");
     zkServer = new ZkTestServer(zkDir);
     zkServer.run();
     
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index deaa4a8..5faa307 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -269,7 +269,7 @@ public class MiniSolrCloudCluster {
 
     this.externalZkServer = zkTestServer != null;
     if (!externalZkServer) {
-      String zkDir = baseDir.resolve("zookeeper/server1/data").toString();
+      Path zkDir = baseDir.resolve("zookeeper/server1/data");
       zkTestServer = new ZkTestServer(zkDir);
       try {
         zkTestServer.run();
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
index 001c76f..eb0ed63 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
@@ -61,6 +61,7 @@ import java.net.UnknownHostException;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SocketChannel;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -90,7 +91,7 @@ public class ZkTestServer {
   
   protected final ZKServerMain zkServer = new ZKServerMain();
 
-  private volatile String zkDir;
+  private volatile Path zkDir;
 
   private volatile int clientPort;
 
@@ -249,6 +250,10 @@ public class ZkTestServer {
           case NodeChildrenChanged:
             childrenLimit.updateForFire(event);
             break;
+          case ChildWatchRemoved:
+            break;
+          case DataWatchRemoved:
+            break;
         }
       }
     }
@@ -259,7 +264,7 @@ public class ZkTestServer {
 
       public TestServerCnxn(ZooKeeperServer zk, SocketChannel sock, SelectionKey sk,
                             NIOServerCnxnFactory factory, WatchLimiter limiter) throws IOException {
-        super(zk, sock, sk, factory);
+        super(zk, sock, sk, factory, null);
         this.limiter = limiter;
       }
 
@@ -278,11 +283,6 @@ public class ZkTestServer {
         super();
         this.limiter = limiter;
       }
-
-      @Override
-      protected NIOServerCnxn createConnection(SocketChannel sock, SelectionKey sk) throws IOException {
-        return new TestServerCnxn(zkServer, sock, sk, this, limiter);
-      }
     }
 
     private class TestZKDatabase extends ZKDatabase {
@@ -331,11 +331,11 @@ public class ZkTestServer {
         // so rather than spawning another thread, we will just call
         // run() in this thread.
         // create a file logger url from the command line args
-        FileTxnSnapLog ftxn = new FileTxnSnapLog(new File(
-            config.getDataLogDir()), new File(config.getDataDir()));
+        FileTxnSnapLog ftxn = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir());
+
         zooKeeperServer = new ZooKeeperServer(ftxn, config.getTickTime(),
             config.getMinSessionTimeout(), config.getMaxSessionTimeout(),
-            null /* this is not used */, new TestZKDatabase(ftxn, limiter));
+            new TestZKDatabase(ftxn, limiter));
         cnxnFactory = new TestServerCnxnFactory(limiter);
         cnxnFactory.configure(config.getClientPortAddress(),
             config.getMaxClientCnxns());
@@ -418,11 +418,11 @@ public class ZkTestServer {
     }
   }
 
-  public ZkTestServer(String zkDir) throws Exception {
+  public ZkTestServer(Path zkDir) throws Exception {
     this(zkDir, 0);
   }
 
-  public ZkTestServer(String zkDir, int port) throws KeeperException, InterruptedException {
+  public ZkTestServer(Path zkDir, int port) throws KeeperException, InterruptedException {
     this.zkDir = zkDir;
     this.clientPort = port;
     String reportAction = System.getProperty("tests.zk.violationReportAction");
@@ -541,8 +541,8 @@ public class ZkTestServer {
 
             {
               setClientPort(ZkTestServer.this.clientPort);
-              this.dataDir = zkDir;
-              this.dataLogDir = zkDir;
+              this.dataDir = zkDir.toFile();
+              this.dataLogDir = zkDir.toFile();
               this.tickTime = theTickTime;
               this.maxSessionTimeout = ZkTestServer.this.maxSessionTimeout;
               this.minSessionTimeout = ZkTestServer.this.minSessionTimeout;
@@ -753,7 +753,7 @@ public class ZkTestServer {
     this.theTickTime = theTickTime;
   }
 
-  public String getZkDir() {
+  public Path getZkDir() {
     return zkDir;
   }