You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2015/07/17 20:53:09 UTC

[5/6] accumulo git commit: ACCUMULO-2346 update all sleeps to a specific time unit

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 5d3e0f0..9563254 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -138,7 +138,6 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.SimpleThreadPool;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -251,6 +250,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.net.HostAndPort;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class TabletServer extends AccumuloServerContext implements Runnable {
   private static final Logger log = LoggerFactory.getLogger(TabletServer.class);
@@ -528,7 +528,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         else if (e.getCause() instanceof TooManyFilesException)
           throw new org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException(scanSession.extent.toThrift());
         else if (e.getCause() instanceof IOException) {
-          UtilWaitThread.sleep(MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS);
+          sleepUninterruptibly(MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS, TimeUnit.MILLISECONDS);
           List<KVEntry> empty = Collections.emptyList();
           bresult = new ScanBatch(empty, true);
           scanSession.nextBatchTask = null;
@@ -1805,7 +1805,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     public void run() {
       while (!majorCompactorDisabled) {
         try {
-          UtilWaitThread.sleep(getConfiguration().getTimeInMillis(Property.TSERV_MAJC_DELAY));
+          sleepUninterruptibly(getConfiguration().getTimeInMillis(Property.TSERV_MAJC_DELAY), TimeUnit.MILLISECONDS);
 
           TreeMap<KeyExtent,Tablet> copyOnlineTablets = new TreeMap<KeyExtent,Tablet>();
 
@@ -1866,7 +1866,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           }
         } catch (Throwable t) {
           log.error("Unexpected exception in " + Thread.currentThread().getName(), t);
-          UtilWaitThread.sleep(1000);
+          sleepUninterruptibly(1, TimeUnit.SECONDS);
         }
       }
     }
@@ -2356,7 +2356,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           return;
         }
         log.info("Waiting for tablet server lock");
-        UtilWaitThread.sleep(5000);
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
       }
       String msg = "Too many retries, exiting.";
       log.info(msg);
@@ -2508,7 +2508,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           }
           returnMasterConnection(iface);
 
-          UtilWaitThread.sleep(1000);
+          sleepUninterruptibly(1, TimeUnit.SECONDS);
         }
       } catch (InterruptedException e) {
         log.info("Interrupt Exception received, shutting down");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 65cbdf1..f42d4a4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.file.blockfile.cache.LruBlockCache;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.NamingThreadFactory;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.FileRef;
@@ -64,6 +63,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  * ResourceManager is responsible for managing the resources of all tablets within a tablet server.
@@ -448,7 +448,7 @@ public class TabletServerResourceManager {
           log.error("Minor compactions for memory managment failed", t);
         }
 
-        UtilWaitThread.sleep(250);
+        sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 304b8c3..e6d7185 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -40,7 +40,6 @@ import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 import org.apache.accumulo.core.util.SimpleThreadPool;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -60,6 +59,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  * Central logging facility for the TServerInfo.
@@ -386,7 +386,7 @@ public class TabletServerLogger {
         if (attempt != 1) {
           log.error("Unexpected error writing to log, retrying attempt " + attempt, t);
         }
-        UtilWaitThread.sleep(100);
+        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
         attempt++;
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index e8e9627..417a107 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -29,6 +29,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -58,7 +59,6 @@ import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.trace.ProbabilitySampler;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -81,6 +81,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class AccumuloReplicaSystem implements ReplicaSystem {
   private static final Logger log = LoggerFactory.getLogger(AccumuloReplicaSystem.class);
@@ -299,7 +300,7 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
           return finalStatus;
         } catch (TTransportException | AccumuloException | AccumuloSecurityException e) {
           log.warn("Could not connect to remote server {}, will retry", peerTserverStr, e);
-          UtilWaitThread.sleep(1000);
+          sleepUninterruptibly(1, TimeUnit.SECONDS);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index 82ae205..6b19b26 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.impl.KeyExtent;
@@ -36,7 +37,6 @@ import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.FileRef;
@@ -51,6 +51,8 @@ import org.apache.accumulo.tserver.TLevel;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 class DatafileManager {
   private final Logger log = Logger.getLogger(DatafileManager.class);
   // access to datafilesizes needs to be synchronized: see CompactionRunner#getNumFiles
@@ -353,7 +355,7 @@ class DatafileManager {
         break;
       } catch (IOException ioe) {
         log.warn("Tablet " + tablet.getExtent() + " failed to rename " + newDatafile + " after MinC, will retry in 60 secs...", ioe);
-        UtilWaitThread.sleep(60 * 1000);
+        sleepUninterruptibly(1, TimeUnit.MINUTES);
       }
     } while (true);
 
@@ -426,7 +428,7 @@ class DatafileManager {
         break;
       } catch (IOException e) {
         log.error("Failed to write to write-ahead log " + e.getMessage() + " will retry", e);
-        UtilWaitThread.sleep(1 * 1000);
+        sleepUninterruptibly(1, TimeUnit.SECONDS);
       }
     } while (true);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
index 490ecd3..3171fcb 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -20,13 +20,13 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.problems.ProblemReport;
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class MinorCompactor extends Compactor {
 
   private static final Logger log = LoggerFactory.getLogger(MinorCompactor.class);
@@ -122,7 +124,7 @@ public class MinorCompactor extends Compactor {
 
         int sleep = sleepTime + random.nextInt(sleepTime);
         log.debug("MinC failed sleeping " + sleep + " ms before retrying");
-        UtilWaitThread.sleep(sleep);
+        sleepUninterruptibly(sleep, TimeUnit.MILLISECONDS);
         sleepTime = (int) Math.round(Math.min(maxSleepTime, sleepTime * growthFactor));
 
         // clean up

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 8fcf035..ad3fb47 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -88,7 +88,6 @@ import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
@@ -150,6 +149,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  *
@@ -1294,7 +1294,7 @@ public class Tablet implements TabletCommitter {
         } catch (RuntimeException t) {
           err = t;
           log.error("Consistency check fails, retrying " + t);
-          UtilWaitThread.sleep(500);
+          sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
         }
       }
       if (err != null) {
@@ -2605,7 +2605,7 @@ public class Tablet implements TabletCommitter {
       }
 
       log.warn("Failed to create dir for tablet in table " + tableId + " in volume " + volume + " + will retry ...");
-      UtilWaitThread.sleep(3000);
+      sleepUninterruptibly(3, TimeUnit.SECONDS);
 
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/shell/src/main/java/org/apache/accumulo/shell/commands/TraceCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/TraceCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/TraceCommand.java
index 86e44e7..4506f28 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/TraceCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/TraceCommand.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.shell.commands;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
@@ -25,13 +26,14 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.BadArgumentException;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.tracer.TraceDump;
 import org.apache.accumulo.tracer.TraceDump.Printer;
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.io.Text;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class TraceCommand extends DebugCommand {
 
   @Override
@@ -74,7 +76,7 @@ public class TraceCommand extends DebugCommand {
             }
             shellState.getReader().println("Waiting for trace information");
             shellState.getReader().flush();
-            UtilWaitThread.sleep(500);
+            sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
           }
           if (traceCount < 0) {
             // display the trace even though there are unrooted spans

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/AssignmentThreadsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/AssignmentThreadsIT.java b/test/src/main/java/org/apache/accumulo/test/AssignmentThreadsIT.java
index 5e60388..48955f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/AssignmentThreadsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/AssignmentThreadsIT.java
@@ -21,18 +21,19 @@ import static org.junit.Assert.assertTrue;
 import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.test.PerformanceTest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // ACCUMULO-1177
 @Category(PerformanceTest.class)
 public class AssignmentThreadsIT extends ConfigurableMacBase {
@@ -85,7 +86,7 @@ public class AssignmentThreadsIT extends ConfigurableMacBase {
     log.info("Taking table offline, again");
     c.tableOperations().offline(tableName, true);
     // wait >10 seconds for thread pool to update
-    UtilWaitThread.sleep(Math.max(0, now + 11 * 1000 - System.currentTimeMillis()));
+    sleepUninterruptibly(Math.max(0, now + 11 * 1000 - System.currentTimeMillis()), TimeUnit.MILLISECONDS);
     now = System.currentTimeMillis();
     log.info("Bringing table back online");
     c.tableOperations().online(tableName, true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 5b0b84d..5dfc136 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
@@ -32,7 +33,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
@@ -48,6 +48,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // Accumulo3047
 public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(BadDeleteMarkersCreatedIT.class);
@@ -158,7 +160,7 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
     c.tableOperations().delete(tableName);
     log.info("Sleeping to let garbage collector run");
     // let gc run
-    UtilWaitThread.sleep(timeoutFactor * 15 * 1000);
+    sleepUninterruptibly(timeoutFactor * 15, TimeUnit.SECONDS);
     log.info("Verifying that delete markers were deleted");
     // look for delete markers
     Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/BalanceFasterIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/BalanceFasterIT.java b/test/src/main/java/org/apache/accumulo/test/BalanceFasterIT.java
index 660051e..15a19a3 100644
--- a/test/src/main/java/org/apache/accumulo/test/BalanceFasterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BalanceFasterIT.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -33,17 +34,17 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.accumulo.test.mrit.IntegrationTestMapReduce;
-import org.apache.accumulo.test.PerformanceTest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // ACCUMULO-2952
 @Category(PerformanceTest.class)
 public class BalanceFasterIT extends ConfigurableMacBase {
@@ -70,7 +71,7 @@ public class BalanceFasterIT extends ConfigurableMacBase {
     }
     conn.tableOperations().addSplits(tableName, splits);
     // give a short wait for balancing
-    UtilWaitThread.sleep(10 * 1000);
+    sleepUninterruptibly(10, TimeUnit.SECONDS);
     // find out where the tabets are
     Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     s.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
index 2474b3e..91b929f 100644
--- a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.test;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -33,7 +34,6 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -46,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class CleanWalIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(CleanWalIT.class);
@@ -119,7 +120,7 @@ public class CleanWalIT extends AccumuloClusterHarness {
     conn.tableOperations().flush(RootTable.NAME, null, null, true);
     try {
       getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      UtilWaitThread.sleep(3 * 1000);
+      sleepUninterruptibly(3, TimeUnit.SECONDS);
     } finally {
       getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
index 74d3593..4f915d3 100644
--- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -79,7 +79,6 @@ import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.FastFormat;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
@@ -97,6 +96,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  *
@@ -652,7 +652,7 @@ public class ConditionalWriterIT extends AccumuloClusterHarness {
     conn.tableOperations().create(tableName);
     conn.tableOperations().addSplits(tableName, nss("2", "4", "6"));
 
-    UtilWaitThread.sleep(2000);
+    sleepUninterruptibly(2, TimeUnit.SECONDS);
 
     int num = 100;
 
@@ -1286,7 +1286,7 @@ public class ConditionalWriterIT extends AccumuloClusterHarness {
     if (!conn.tableOperations().exists("trace")) {
       tracer = mac.exec(TraceServer.class);
       while (!conn.tableOperations().exists("trace")) {
-        UtilWaitThread.sleep(1000);
+        sleepUninterruptibly(1, TimeUnit.SECONDS);
       }
     }
 
@@ -1294,7 +1294,7 @@ public class ConditionalWriterIT extends AccumuloClusterHarness {
     conn.tableOperations().create(tableName);
 
     DistributedTrace.enable("localhost", "testTrace", mac.getClientConfig());
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
     Span root = Trace.on("traceTest");
     ConditionalWriter cw = conn.createConditionalWriter(tableName, new ConditionalWriterConfig());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
index 52d2086..d4f4d58 100644
--- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -25,6 +25,7 @@ import java.io.OutputStream;
 import java.util.Collection;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -38,7 +39,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -50,6 +50,8 @@ import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class ExistingMacIT extends ConfigurableMacBase {
   @Override
   public int defaultTimeoutSeconds() {
@@ -101,7 +103,7 @@ public class ExistingMacIT extends ConfigurableMacBase {
     }
 
     // TODO clean out zookeeper? following sleep waits for ephemeral nodes to go away
-    UtilWaitThread.sleep(10000);
+    sleepUninterruptibly(10, TimeUnit.SECONDS);
 
     File hadoopConfDir = createTestDir(ExistingMacIT.class.getSimpleName() + "_hadoop_conf");
     FileUtils.deleteQuietly(hadoopConfDir);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
index 1c3d89e..0f00caf 100644
--- a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
+++ b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
@@ -25,6 +25,7 @@ import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -42,10 +43,10 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.Stat;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
  *
@@ -180,7 +181,7 @@ public class IMMLGBenchmark {
 
       conn.tableOperations().setLocalityGroups(table, groups);
       conn.tableOperations().offline(table);
-      UtilWaitThread.sleep(1000);
+      sleepUninterruptibly(1, TimeUnit.SECONDS);
       conn.tableOperations().online(table);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
index 84a5996..73714c5 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -35,7 +36,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.accumulo.test.functional.SlowIterator;
@@ -43,6 +43,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class MetaGetsReadersIT extends ConfigurableMacBase {
 
@@ -98,7 +99,7 @@ public class MetaGetsReadersIT extends ConfigurableMacBase {
     t1.start();
     Thread t2 = slowScan(c, tableName, stop);
     t2.start();
-    UtilWaitThread.sleep(500);
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
     long now = System.currentTimeMillis();
     Scanner m = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     Iterators.size(m.iterator());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
index 37e4957..d584613 100644
--- a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -32,7 +33,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class MultiTableRecoveryIT extends ConfigurableMacBase {
 
@@ -116,7 +117,7 @@ public class MultiTableRecoveryIT extends ConfigurableMacBase {
         try {
           int i = 0;
           while (!stop.get()) {
-            UtilWaitThread.sleep(10 * 1000);
+            sleepUninterruptibly(10, TimeUnit.SECONDS);
             System.out.println("Restarting");
             getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
             getCluster().start();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
index 0ecdd0d..b407ead 100644
--- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
@@ -34,6 +34,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -74,7 +75,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
@@ -83,6 +83,8 @@ import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // Testing default namespace configuration with inheritance requires altering the system state and restoring it back to normal
 // Punt on this for now and just let it use a minicluster.
 public class NamespacesIT extends AccumuloClusterHarness {
@@ -329,7 +331,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
     // verify entry is filtered out (also, verify conflict checking API)
     c.namespaceOperations().checkIteratorConflicts(namespace, setting, EnumSet.allOf(IteratorScope.class));
     c.namespaceOperations().attachIterator(namespace, setting);
-    UtilWaitThread.sleep(2 * 1000);
+    sleepUninterruptibly(2, TimeUnit.SECONDS);
     try {
       c.namespaceOperations().checkIteratorConflicts(namespace, setting, EnumSet.allOf(IteratorScope.class));
       fail();
@@ -345,7 +347,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
 
     // verify can see inserted entry again
     c.namespaceOperations().removeIterator(namespace, setting.getName(), EnumSet.allOf(IteratorScope.class));
-    UtilWaitThread.sleep(2 * 1000);
+    sleepUninterruptibly(2, TimeUnit.SECONDS);
     assertFalse(c.namespaceOperations().listIterators(namespace).containsKey(iterName));
     assertFalse(c.tableOperations().listIterators(t1).containsKey(iterName));
     s = c.createScanner(t1, Authorizations.EMPTY);
@@ -485,7 +487,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
     int num = c.namespaceOperations().listConstraints(namespace).get(constraintClassName);
     assertEquals(num, (int) c.tableOperations().listConstraints(t1).get(constraintClassName));
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    UtilWaitThread.sleep(250);
+    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
 
     Mutation m1 = new Mutation("r1");
     Mutation m2 = new Mutation("r2");
@@ -506,7 +508,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
     assertFalse(c.namespaceOperations().listConstraints(namespace).containsKey(constraintClassName));
     assertFalse(c.tableOperations().listConstraints(t1).containsKey(constraintClassName));
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    UtilWaitThread.sleep(250);
+    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
 
     bw = c.createBatchWriter(t1, new BatchWriterConfig());
     bw.addMutations(Arrays.asList(m1, m2, m3));
@@ -817,7 +819,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
     // set the filter, verify that accumulo namespace is the only one unaffected
     c.instanceOperations().setProperty(k, v);
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    UtilWaitThread.sleep(250);
+    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(v));
     assertEquals(systemNamespaceShouldInherit, checkNamespaceHasProp(Namespaces.ACCUMULO_NAMESPACE, k, v));
     assertEquals(systemNamespaceShouldInherit, checkTableHasProp(RootTable.NAME, k, v));
@@ -830,7 +832,7 @@ public class NamespacesIT extends AccumuloClusterHarness {
     // verify it is no longer inherited
     c.instanceOperations().removeProperty(k);
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    UtilWaitThread.sleep(250);
+    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
     assertFalse(c.instanceOperations().getSystemConfiguration().containsValue(v));
     assertFalse(checkNamespaceHasProp(Namespaces.ACCUMULO_NAMESPACE, k, v));
     assertFalse(checkTableHasProp(RootTable.NAME, k, v));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/NativeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/NativeMapPerformanceTest.java b/test/src/main/java/org/apache/accumulo/test/NativeMapPerformanceTest.java
index 0285092..b5293dc 100644
--- a/test/src/main/java/org/apache/accumulo/test/NativeMapPerformanceTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/NativeMapPerformanceTest.java
@@ -25,15 +25,17 @@ import java.util.Random;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.FastFormat;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.tserver.NativeMap;
 import org.apache.hadoop.io.Text;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class NativeMapPerformanceTest {
 
   private static final byte ROW_PREFIX[] = new byte[] {'r'};
@@ -170,7 +172,7 @@ public class NativeMapPerformanceTest {
     System.gc();
     System.gc();
 
-    UtilWaitThread.sleep(3000);
+    sleepUninterruptibly(3, TimeUnit.SECONDS);
 
     System.out.printf("mapType:%10s   put rate:%,6.2f  scan rate:%,6.2f  get rate:%,6.2f  delete time : %6.2f  mem : %,d%n", "" + mapType, (numRows * numCols)
         / ((tpe - tps) / 1000.0), (size) / ((tie - tis) / 1000.0), numLookups / ((tge - tgs) / 1000.0), (tde - tds) / 1000.0, memUsed);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index cf22df5..b51e02b 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -38,6 +38,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import jline.console.ConsoleReader;
 
@@ -61,7 +62,6 @@ import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.test.functional.SlowIterator;
@@ -87,6 +87,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ShellServerIT extends SharedMiniClusterBase {
   public static class TestOutputStream extends OutputStream {
@@ -261,7 +262,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
 
     // give the tracer some time to start
     while (!tops.exists("trace")) {
-      UtilWaitThread.sleep(1000);
+      sleepUninterruptibly(1, TimeUnit.SECONDS);
     }
   }
 
@@ -704,7 +705,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
         ts.exec("getauths", true, "bar", true);
         passed = true;
       } catch (Exception e) {
-        UtilWaitThread.sleep(300);
+        sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
       }
     }
     assertTrue("Could not successfully see updated authoriations", passed);
@@ -1000,7 +1001,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", true);
     ts.exec("constraint -t " + table + " -d 2", true, "Removed constraint 2 from table " + table);
     // wait for zookeeper updates to propagate
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
     ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", false);
     ts.exec("deletetable -f " + table);
   }
@@ -1196,7 +1197,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("scan -b 02", true, "value", false);
     ts.exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true);
     // Need to allow time for this to propagate through zoocache/zookeeper
-    UtilWaitThread.sleep(3000);
+    sleepUninterruptibly(3, TimeUnit.SECONDS);
 
     ts.exec("interpreter -l", true, "HexScan", true);
     ts.exec("scan -b 02", true, "value", true);
@@ -1265,7 +1266,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
       // wait for both tservers to start up
       if (ts.output.get().split("\n").length == 3)
         break;
-      UtilWaitThread.sleep(1000);
+      sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     }
     assertEquals(3, ts.output.get().split("\n").length);
@@ -1347,7 +1348,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
           log.info("Ignoring scan because of wrong table: " + currentScan);
         }
       }
-      UtilWaitThread.sleep(300);
+      sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
     }
     thread.join();
 
@@ -1391,7 +1392,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     ts.exec("createtable " + table, true);
     ts.exec("config -t " + table + " -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true);
 
-    UtilWaitThread.sleep(200);
+    sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 
     // We can't use the setiter command as Filter implements OptionDescriber which
     // forces us to enter more input that I don't know how to input
@@ -1400,7 +1401,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
 
     ts.exec("insert foo f q v", true);
 
-    UtilWaitThread.sleep(100);
+    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
     ts.exec("scan -np", true, "foo", false);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java b/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
index 4cad3a7..fc54b64 100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitCancelsMajCIT.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.EnumSet;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -30,12 +31,13 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // ACCUMULO-2862
 public class SplitCancelsMajCIT extends SharedMiniClusterBase {
 
@@ -75,7 +77,7 @@ public class SplitCancelsMajCIT extends SharedMiniClusterBase {
     thread.start();
 
     long now = System.currentTimeMillis();
-    UtilWaitThread.sleep(10 * 1000);
+    sleepUninterruptibly(10, TimeUnit.SECONDS);
     // split the table, interrupts the compaction
     SortedSet<Text> partitionKeys = new TreeSet<Text>();
     partitionKeys.add(new Text("10"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
index 298c761..3f8c30c 100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.test;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -35,12 +36,12 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class SplitRecoveryIT extends AccumuloClusterHarness {
 
@@ -81,7 +82,7 @@ public class SplitRecoveryIT extends AccumuloClusterHarness {
       // take the table offline
       connector.tableOperations().offline(tableName);
       while (!isOffline(tableName, connector))
-        UtilWaitThread.sleep(200);
+        sleepUninterruptibly(200, TimeUnit.MILLISECONDS);
 
       // poke a partial split into the metadata table
       connector.securityOperations().grantTablePermission(getAdminPrincipal(), MetadataTable.NAME, TablePermission.WRITE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
index 789b089..7d6e6bd 100644
--- a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
@@ -34,6 +34,7 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -55,7 +56,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.functional.BadIterator;
 import org.apache.hadoop.io.Text;
@@ -66,6 +66,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class TableOperationsIT extends AccumuloClusterHarness {
 
@@ -361,7 +362,7 @@ public class TableOperationsIT extends AccumuloClusterHarness {
     List<IteratorSetting> list = new ArrayList<>();
     list.add(new IteratorSetting(15, BadIterator.class));
     connector.tableOperations().compact(tableName, null, null, list, true, false); // don't block
-    UtilWaitThread.sleep(2000); // start compaction
+    sleepUninterruptibly(2, TimeUnit.SECONDS); // start compaction
     connector.tableOperations().cancelCompaction(tableName);
 
     Scanner scanner = connector.createScanner(tableName, Authorizations.EMPTY);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
index 06bf394..944b310 100644
--- a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
@@ -19,17 +19,19 @@ package org.apache.accumulo.test;
 import static org.junit.Assert.assertEquals;
 
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 // ACCUMULO-2480
 public class TabletServerGivesUpIT extends ConfigurableMacBase {
 
@@ -66,7 +68,7 @@ public class TabletServerGivesUpIT extends ConfigurableMacBase {
     splitter.start();
     // wait for the tserver to give up on writing to the WAL
     while (conn.instanceOperations().getTabletServers().size() == 1) {
-      UtilWaitThread.sleep(1000);
+      sleepUninterruptibly(1, TimeUnit.SECONDS);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
index bf2e7f1..be800ad 100644
--- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
@@ -29,7 +29,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.AccumuloServerContext;
@@ -39,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
 import com.google.common.net.HostAndPort;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 // see ACCUMULO-1950
 public class TotalQueuedIT extends ConfigurableMacBase {
@@ -63,7 +63,7 @@ public class TotalQueuedIT extends ConfigurableMacBase {
     c.tableOperations().create(tableName);
     c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "9999");
     c.tableOperations().setProperty(tableName, Property.TABLE_FILE_MAX.getKey(), "999");
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
     // get an idea of how fast the syncs occur
     byte row[] = new byte[250];
     BatchWriterConfig cfg = new BatchWriterConfig();
@@ -94,7 +94,7 @@ public class TotalQueuedIT extends ConfigurableMacBase {
     // Now with a much bigger total queue
     c.instanceOperations().setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "" + LARGE_QUEUE_SIZE);
     c.tableOperations().flush(tableName, null, null, true);
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
     bw = c.createBatchWriter(tableName, cfg);
     now = System.currentTimeMillis();
     bytesSent = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java
index 1c6e3df..72f87aa 100644
--- a/test/src/main/java/org/apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TracerRecoversAfterOfflineTableIT.java
@@ -18,6 +18,8 @@ package org.apache.accumulo.test;
 
 import static org.junit.Assert.assertTrue;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -27,7 +29,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
@@ -38,6 +39,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 /**
  *
  */
@@ -61,9 +64,9 @@ public class TracerRecoversAfterOfflineTableIT extends ConfigurableMacBase {
       MiniAccumuloClusterImpl mac = cluster;
       tracer = mac.exec(TraceServer.class);
       while (!conn.tableOperations().exists("trace")) {
-        UtilWaitThread.sleep(1000);
+        sleepUninterruptibly(1, TimeUnit.SECONDS);
       }
-      UtilWaitThread.sleep(5000);
+      sleepUninterruptibly(5, TimeUnit.SECONDS);
     }
 
     log.info("Taking table offline");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
index 2c32176..dc77f37 100644
--- a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
+++ b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousBatchWalker.java
@@ -35,11 +35,11 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.validators.PositiveInteger;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ContinuousBatchWalker {
 
@@ -76,7 +76,7 @@ public class ContinuousBatchWalker {
 
       runBatchScan(scanOpts.scanBatchSize, bs, batch, ranges);
 
-      UtilWaitThread.sleep(opts.sleepTime);
+      sleepUninterruptibly(opts.sleepTime, TimeUnit.MILLISECONDS);
     }
 
   }
@@ -157,7 +157,7 @@ public class ContinuousBatchWalker {
 
       System.out.println("FSB " + t1 + " " + (t2 - t1) + " " + count);
 
-      UtilWaitThread.sleep(100);
+      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
     }
 
     HashSet<Text> ret = new HashSet<Text>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousScanner.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousScanner.java b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousScanner.java
index a77de3d..63709df 100644
--- a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousScanner.java
+++ b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousScanner.java
@@ -21,6 +21,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.cli.ScannerOpts;
@@ -30,11 +31,11 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.validators.PositiveInteger;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ContinuousScanner {
 
@@ -100,7 +101,7 @@ public class ContinuousScanner {
       System.out.printf("SCN %d %s %d %d%n", t1, new String(scanStart, UTF_8), (t2 - t1), count);
 
       if (opts.sleepTime > 0)
-        UtilWaitThread.sleep(opts.sleepTime);
+        sleepUninterruptibly(opts.sleepTime, TimeUnit.MILLISECONDS);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
index 118f053..1d38126 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -42,7 +43,6 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -52,6 +52,8 @@ import org.apache.hadoop.mapreduce.Job;
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
   AccumuloInputFormat inputFormat;
@@ -108,7 +110,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
     for (int i = 0; i < 10000; i += 1000)
       splitsToAdd.add(new Text(String.format("%09d", i)));
     conn.tableOperations().addSplits(table, splitsToAdd);
-    UtilWaitThread.sleep(500); // wait for splits to be propagated
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS); // wait for splits to be propagated
 
     // get splits without setting any range
     Collection<Text> actualSplits = conn.tableOperations().listSplits(table);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
index 4b4aeac..ddaca1e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,11 +35,12 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class AddSplitIT extends AccumuloClusterHarness {
 
   @Override
@@ -61,7 +63,7 @@ public class AddSplitIT extends AccumuloClusterHarness {
 
     c.tableOperations().addSplits(tableName, splits);
 
-    UtilWaitThread.sleep(100);
+    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
     Collection<Text> actualSplits = c.tableOperations().listSplits(tableName);
 
@@ -81,7 +83,7 @@ public class AddSplitIT extends AccumuloClusterHarness {
 
     c.tableOperations().addSplits(tableName, splits);
 
-    UtilWaitThread.sleep(100);
+    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
     actualSplits = c.tableOperations().listSplits(tableName);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
index 4c6fc00..c730f9b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
@@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 
 import java.util.EnumSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -30,12 +31,12 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class BadIteratorMincIT extends AccumuloClusterHarness {
 
@@ -61,7 +62,7 @@ public class BadIteratorMincIT extends AccumuloClusterHarness {
     bw.close();
 
     c.tableOperations().flush(tableName, null, null, false);
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     // minc should fail, so there should be no files
     FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);
@@ -74,7 +75,7 @@ public class BadIteratorMincIT extends AccumuloClusterHarness {
     // remove the bad iterator
     c.tableOperations().removeIterator(tableName, BadIterator.class.getSimpleName(), EnumSet.of(IteratorScope.minc));
 
-    UtilWaitThread.sleep(5000);
+    sleepUninterruptibly(5, TimeUnit.SECONDS);
 
     // minc should complete
     FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 1, 1);
@@ -93,12 +94,12 @@ public class BadIteratorMincIT extends AccumuloClusterHarness {
     bw.close();
 
     // make sure property is given time to propagate
-    UtilWaitThread.sleep(500);
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
 
     c.tableOperations().flush(tableName, null, null, false);
 
     // make sure the flush has time to start
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     // this should not hang
     c.tableOperations().delete(tableName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
index 14295c4..7d36cd2 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map.Entry;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -34,7 +35,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -43,6 +43,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class BatchScanSplitIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(BatchScanSplitIT.class);
 
@@ -80,7 +82,7 @@ public class BatchScanSplitIT extends AccumuloClusterHarness {
 
     Collection<Text> splits = getConnector().tableOperations().listSplits(tableName);
     while (splits.size() < 2) {
-      UtilWaitThread.sleep(1);
+      sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
       splits = getConnector().tableOperations().listSplits(tableName);
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
index 7c05a0f..353a6b9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
@@ -36,12 +36,12 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class BatchWriterFlushIT extends AccumuloClusterHarness {
 
@@ -74,7 +74,7 @@ public class BatchWriterFlushIT extends AccumuloClusterHarness {
     m.put(new Text("cf"), new Text("cq"), new Value("1".getBytes(UTF_8)));
     bw.addMutation(m);
 
-    UtilWaitThread.sleep(500);
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
 
     int count = Iterators.size(scanner.iterator());
 
@@ -82,7 +82,7 @@ public class BatchWriterFlushIT extends AccumuloClusterHarness {
       throw new Exception("Flushed too soon");
     }
 
-    UtilWaitThread.sleep(1500);
+    sleepUninterruptibly(1500, TimeUnit.MILLISECONDS);
 
     count = Iterators.size(scanner.iterator());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 226f8f1..bf0b618 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -18,6 +18,8 @@ package org.apache.accumulo.test.functional;
 
 import static com.google.common.base.Charsets.UTF_8;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.ClientConfiguration;
@@ -26,7 +28,6 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -40,6 +41,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 /**
  * This test verifies that when a lot of files are bulk imported into a table with one tablet and then splits that not all map files go to the children tablets.
  */
@@ -103,11 +106,11 @@ public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
     // initiate splits
     getConnector().tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "100K");
 
-    UtilWaitThread.sleep(2000);
+    sleepUninterruptibly(2, TimeUnit.SECONDS);
 
     // wait until over split threshold -- should be 78 splits
     while (getConnector().tableOperations().listSplits(tableName).size() < 75) {
-      UtilWaitThread.sleep(500);
+      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
     }
 
     FunctionalTestUtils.checkSplits(c, tableName, 50, 100);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
index fd8ba2c..7935aa4 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
@@ -25,11 +25,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class CacheTestReader {
   public static void main(String[] args) throws Exception {
     String rootDir = args[0];
@@ -77,7 +79,7 @@ public class CacheTestReader {
       fos.close();
       oos.close();
 
-      UtilWaitThread.sleep(20);
+      sleepUninterruptibly(20, TimeUnit.MILLISECONDS);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
index 76e8168..5d05a71 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
@@ -27,13 +27,15 @@ import java.util.Map;
 import java.util.Random;
 import java.util.TreeMap;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class CacheTestWriter {
 
   static final int NUM_DATA = 3;
@@ -156,7 +158,7 @@ public class CacheTestWriter {
             break;
         }
 
-        UtilWaitThread.sleep(5);
+        sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
index acbb699..29f2780 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -38,7 +39,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Combiner;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -49,6 +49,8 @@ import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class ClassLoaderIT extends AccumuloClusterHarness {
 
   private static final long ZOOKEEPER_PROPAGATION_TIME = 10 * 1000;
@@ -94,15 +96,15 @@ public class ClassLoaderIT extends AccumuloClusterHarness {
     FileSystem fs = getCluster().getFileSystem();
     Path jarPath = new Path(rootPath + "/lib/ext/Test.jar");
     copyStreamToFileSystem(fs, "/TestCombinerX.jar", jarPath);
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
     IteratorSetting is = new IteratorSetting(10, "TestCombiner", "org.apache.accumulo.test.functional.TestCombiner");
     Combiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("cf")));
     c.tableOperations().attachIterator(tableName, is, EnumSet.of(IteratorScope.scan));
-    UtilWaitThread.sleep(ZOOKEEPER_PROPAGATION_TIME);
+    sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
     scanCheck(c, tableName, "TestX");
     fs.delete(jarPath, true);
     copyStreamToFileSystem(fs, "/TestCombinerY.jar", jarPath);
-    UtilWaitThread.sleep(5000);
+    sleepUninterruptibly(5, TimeUnit.SECONDS);
     scanCheck(c, tableName, "TestY");
     fs.delete(jarPath, true);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
index 75eecfd..d462b53 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
@@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.util.EnumSet;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -36,7 +37,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -44,6 +44,7 @@ import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ConcurrencyIT extends AccumuloClusterHarness {
 
@@ -117,7 +118,7 @@ public class ConcurrencyIT extends AccumuloClusterHarness {
     ScanTask st1 = new ScanTask(c, tableName, 100);
     st1.start();
 
-    UtilWaitThread.sleep(50);
+    sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
     c.tableOperations().flush(tableName, null, null, true);
 
     for (int i = 0; i < 50; i++) {
@@ -142,7 +143,7 @@ public class ConcurrencyIT extends AccumuloClusterHarness {
     ScanTask st3 = new ScanTask(c, tableName, 150);
     st3.start();
 
-    UtilWaitThread.sleep(50);
+    sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
     c.tableOperations().flush(tableName, null, null, false);
 
     st3.join();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
index 4ef4a61..5817b03 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -36,7 +37,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -45,6 +45,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 public class ConstraintIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(ConstraintIT.class);
 
@@ -155,7 +157,7 @@ public class ConstraintIT extends AccumuloClusterHarness {
 
     // remove the numeric value constraint
     getConnector().tableOperations().removeConstraint(tableName, 2);
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     // now should be able to add a non numeric value
     bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
@@ -178,7 +180,7 @@ public class ConstraintIT extends AccumuloClusterHarness {
 
     // add a constraint that references a non-existant class
     getConnector().tableOperations().setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX + "1", "com.foobar.nonExistantClass");
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     // add a mutation
     bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());
@@ -218,7 +220,7 @@ public class ConstraintIT extends AccumuloClusterHarness {
 
     // remove the bad constraint
     getConnector().tableOperations().removeConstraint(tableName, 1);
-    UtilWaitThread.sleep(1000);
+    sleepUninterruptibly(1, TimeUnit.SECONDS);
 
     // try the mutation again
     bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ec6291a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
index 2650c89..4577813 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
@@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -31,7 +32,6 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -42,6 +42,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class DeleteEverythingIT extends AccumuloClusterHarness {
 
@@ -103,7 +104,7 @@ public class DeleteEverythingIT extends AccumuloClusterHarness {
     getConnector().tableOperations().flush(tableName, null, null, true);
 
     getConnector().tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "1.0");
-    UtilWaitThread.sleep(4000);
+    sleepUninterruptibly(4, TimeUnit.SECONDS);
 
     FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);