You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2019/04/10 22:33:36 UTC

[accumulo] branch master updated: Make Monitor service non-static (#1083)

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

ctubbsii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new f1463bf  Make Monitor service non-static (#1083)
f1463bf is described below

commit f1463bfbf54df82e4b9508742426bf4b3fb088e2
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed Apr 10 18:33:32 2019 -0400

    Make Monitor service non-static (#1083)
    
    * Convert all static fields/methods in Monitor to instance members
    (except for constants)
    * Use dependency injection to wire the Monitor through Jetty and into
    the Jersey application resources
    * Remove unnecessary code to asynchronously get the instanceName (it's
    available on server startup, and can be obtained from the ServerContext)
    * Ensure the Monitor is properly instantiated with the ServerContext
    * Simplify fetching lock mechanism (using a simple AtomicBoolean instead
    of synchronization)
    * Simplify some Monitor internals, especially those pertaining to
    initializing container resources by inlining MaxList into an anonymous
    subclass of LinkedList that is constructed by a convenience method; this
    makes some of the many Monitor members more readable by reducing the
    number of lines which are wrapped to construct these lists
    * Remove unnecessary members and group static constants together at the
    top of the class
---
 server/monitor/pom.xml                             |  14 +
 .../apache/accumulo/monitor/EmbeddedWebServer.java |   6 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  | 373 ++++++++++-----------
 .../apache/accumulo/monitor/rest/XMLResource.java  |  10 +-
 .../rest/bulkImports/BulkImportResource.java       |   8 +-
 .../monitor/rest/gc/GarbageCollectorResource.java  |  18 +-
 .../monitor/rest/master/MasterResource.java        |  68 ++--
 .../monitor/rest/problems/ProblemsResource.java    |  29 +-
 .../rest/replication/ReplicationResource.java      |  10 +-
 .../accumulo/monitor/rest/scans/ScansResource.java |   8 +-
 .../rest/statistics/StatisticsResource.java        |  50 +--
 .../monitor/rest/status/StatusResource.java        |  20 +-
 .../monitor/rest/tables/TablesResource.java        |  48 ++-
 .../monitor/rest/trace/TracesResource.java         |  10 +-
 .../monitor/rest/tservers/TabletServer.java        |  15 +-
 .../rest/tservers/TabletServerInformation.java     |   9 +-
 .../rest/tservers/TabletServerResource.java        |  42 ++-
 .../org/apache/accumulo/monitor/view/WebViews.java |  15 +-
 .../org/apache/accumulo/monitor/it/WebViewsIT.java |  38 ++-
 .../rest/tservers/TabletServerInformationTest.java |   6 +-
 20 files changed, 435 insertions(+), 362 deletions(-)

diff --git a/server/monitor/pom.xml b/server/monitor/pom.xml
index 458098a..bef5c14 100644
--- a/server/monitor/pom.xml
+++ b/server/monitor/pom.xml
@@ -101,6 +101,20 @@
       <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-api</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.inject</groupId>
+          <artifactId>javax.inject</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.hk2.external</groupId>
+      <artifactId>javax.inject</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.glassfish.jersey.containers</groupId>
       <artifactId>jersey-container-jetty-http</artifactId>
     </dependency>
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java
index 18cde9f..d67312b 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java
@@ -39,11 +39,11 @@ public class EmbeddedWebServer {
   private final ServerConnector connector;
   private final ServletContextHandler handler;
 
-  public EmbeddedWebServer(String host, int port) {
+  public EmbeddedWebServer(Monitor monitor, int port) {
     server = new Server();
-    final AccumuloConfiguration conf = Monitor.getContext().getConfiguration();
+    final AccumuloConfiguration conf = monitor.getContext().getConfiguration();
     connector = new ServerConnector(server, getConnectionFactories(conf));
-    connector.setHost(host);
+    connector.setHost(monitor.getContext().getHostname());
     connector.setPort(port);
 
     handler = new ServletContextHandler(
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index b23d47d..3cf3ffb 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -32,11 +32,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.TimerTask;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
+
+import javax.inject.Singleton;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.MasterClient;
@@ -66,17 +66,17 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerOpts;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.monitor.LogService;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.server.util.Halt;
 import org.apache.accumulo.server.util.TableInfoUtil;
-import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.resource.Resource;
+import org.glassfish.hk2.api.Factory;
+import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.jackson.JacksonFeature;
 import org.glassfish.jersey.logging.LoggingFeature;
 import org.glassfish.jersey.server.ResourceConfig;
@@ -89,90 +89,76 @@ import org.slf4j.LoggerFactory;
 /**
  * Serve master statistics with an embedded web server.
  */
-public class Monitor implements HighlyAvailableService {
-  private static final Logger log = LoggerFactory.getLogger(Monitor.class);
+public class Monitor implements Runnable, HighlyAvailableService {
 
+  private static final Logger log = LoggerFactory.getLogger(Monitor.class);
   private static final int REFRESH_TIME = 5;
-  private static AtomicLong lastRecalc = new AtomicLong(0L);
-  private static double totalIngestRate = 0.0;
-  private static double totalQueryRate = 0.0;
-  private static double totalScanRate = 0.0;
-  private static long totalEntries = 0L;
-  private static int totalTabletCount = 0;
-  private static long totalHoldTime = 0;
-  private static long totalLookups = 0;
-  private static int totalTables = 0;
-  public static HighlyAvailableService HA_SERVICE_INSTANCE = null;
-  private static final AtomicBoolean monitorInitialized = new AtomicBoolean(false);
-
-  private static class MaxList<T> extends LinkedList<Pair<Long,T>> {
-    private static final long serialVersionUID = 1L;
-
-    private long maxDelta;
-
-    public MaxList(long maxDelta) {
-      this.maxDelta = maxDelta;
-    }
 
-    @Override
-    public boolean add(Pair<Long,T> obj) {
-      boolean result = super.add(obj);
+  private final ServerContext context;
+  private final long START_TIME;
 
-      if (obj.getFirst() - get(0).getFirst() > maxDelta)
-        remove(0);
+  Monitor(ServerContext context) {
+    this.context = context;
+    START_TIME = System.currentTimeMillis();
+  }
 
-      return result;
-    }
+  private final AtomicLong lastRecalc = new AtomicLong(0L);
+  private double totalIngestRate = 0.0;
+  private double totalQueryRate = 0.0;
+  private double totalScanRate = 0.0;
+  private long totalEntries = 0L;
+  private int totalTabletCount = 0;
+  private long totalHoldTime = 0;
+  private long totalLookups = 0;
+  private int totalTables = 0;
+  private final AtomicBoolean monitorInitialized = new AtomicBoolean(false);
+
+  private static <T> List<Pair<Long,T>> newMaxList() {
+    return Collections.synchronizedList(new LinkedList<Pair<Long,T>>() {
+
+      private static final long serialVersionUID = 1L;
+      private final long maxDelta = 60 * 60 * 1000;
+
+      @Override
+      public boolean add(Pair<Long,T> obj) {
+        boolean result = super.add(obj);
+        if (obj.getFirst() - get(0).getFirst() > maxDelta) {
+          remove(0);
+        }
+        return result;
+      }
 
+    });
   }
 
-  private static final int MAX_TIME_PERIOD = 60 * 60 * 1000;
-  private static final List<Pair<Long,Double>> loadOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> ingestRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> ingestByteRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Integer>> minorCompactionsOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Integer>> majorCompactionsOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> lookupsOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Integer>> queryRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Integer>> scanRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> queryByteRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> indexCacheHitRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static final List<Pair<Long,Double>> dataCacheHitRateOverTime = Collections
-      .synchronizedList(new MaxList<>(MAX_TIME_PERIOD));
-  private static EventCounter lookupRateTracker = new EventCounter();
-  private static EventCounter indexCacheHitTracker = new EventCounter();
-  private static EventCounter indexCacheRequestTracker = new EventCounter();
-  private static EventCounter dataCacheHitTracker = new EventCounter();
-  private static EventCounter dataCacheRequestTracker = new EventCounter();
-
-  private static volatile boolean fetching = false;
-  private static MasterMonitorInfo mmi;
-  private static Map<TableId,Map<ProblemType,Integer>> problemSummary = Collections.emptyMap();
-  private static Exception problemException;
-  private static GCStatus gcStatus;
-
-  private static ServerConfigurationFactory config;
-  private static ServerContext context;
-
-  private static EmbeddedWebServer server;
+  private final List<Pair<Long,Double>> loadOverTime = newMaxList();
+  private final List<Pair<Long,Double>> ingestRateOverTime = newMaxList();
+  private final List<Pair<Long,Double>> ingestByteRateOverTime = newMaxList();
+  private final List<Pair<Long,Integer>> minorCompactionsOverTime = newMaxList();
+  private final List<Pair<Long,Integer>> majorCompactionsOverTime = newMaxList();
+  private final List<Pair<Long,Double>> lookupsOverTime = newMaxList();
+  private final List<Pair<Long,Integer>> queryRateOverTime = newMaxList();
+  private final List<Pair<Long,Integer>> scanRateOverTime = newMaxList();
+  private final List<Pair<Long,Double>> queryByteRateOverTime = newMaxList();
+  private final List<Pair<Long,Double>> indexCacheHitRateOverTime = newMaxList();
+  private final List<Pair<Long,Double>> dataCacheHitRateOverTime = newMaxList();
+  private EventCounter lookupRateTracker = new EventCounter();
+  private EventCounter indexCacheHitTracker = new EventCounter();
+  private EventCounter indexCacheRequestTracker = new EventCounter();
+  private EventCounter dataCacheHitTracker = new EventCounter();
+  private EventCounter dataCacheRequestTracker = new EventCounter();
+
+  private final AtomicBoolean fetching = new AtomicBoolean(false);
+  private MasterMonitorInfo mmi;
+  private Map<TableId,Map<ProblemType,Integer>> problemSummary = Collections.emptyMap();
+  private Exception problemException;
+  private GCStatus gcStatus;
+
+  private EmbeddedWebServer server;
 
   private ZooLock monitorLock;
 
-  private static final String DEFAULT_INSTANCE_NAME = "(Unavailable)";
-  public static final AtomicReference<String> cachedInstanceName = new AtomicReference<>(
-      DEFAULT_INSTANCE_NAME);
-
-  private static class EventCounter {
+  private class EventCounter {
 
     Map<String,Pair<Long,Long>> prevSamples = new HashMap<>();
     Map<String,Pair<Long,Long>> samples = new HashMap<>();
@@ -229,7 +215,7 @@ public class Monitor implements HighlyAvailableService {
     }
   }
 
-  public static void fetchData() {
+  public void fetchData() {
     double totalIngestRate = 0.;
     double totalIngestByteRate = 0.;
     double totalQueryRate = 0.;
@@ -243,34 +229,16 @@ public class Monitor implements HighlyAvailableService {
 
     // only recalc every so often
     long currentTime = System.currentTimeMillis();
-    if (currentTime - lastRecalc.get() < REFRESH_TIME * 1000)
+    if (currentTime - lastRecalc.get() < REFRESH_TIME * 1000) {
       return;
-
-    synchronized (Monitor.class) {
-      // Learn our instance name asynchronously so we don't hang up if zookeeper is down
-      if (cachedInstanceName.get().equals(DEFAULT_INSTANCE_NAME)) {
-        SimpleTimer.getInstance(config.getSystemConfiguration()).schedule(new TimerTask() {
-          @Override
-          public void run() {
-            synchronized (Monitor.class) {
-              if (cachedInstanceName.get().equals(DEFAULT_INSTANCE_NAME)) {
-                final String instanceName = context.getInstanceName();
-                if (instanceName != null) {
-                  cachedInstanceName.set(instanceName);
-                }
-              }
-            }
-          }
-        }, 0);
-      }
     }
 
-    synchronized (Monitor.class) {
-      if (fetching)
-        return;
-      fetching = true;
+    // try to begin fetching; return if unsuccessful (because another thread is already fetching)
+    if (!fetching.compareAndSet(false, true)) {
+      return;
     }
-
+    // DO NOT ADD CODE HERE that could throw an exception before we enter the try block
+    // Otherwise, we'll never release the lock by unsetting 'fetching' in the the finally block
     try {
       while (retry) {
         MasterClientService.Iface client = null;
@@ -282,7 +250,7 @@ public class Monitor implements HighlyAvailableService {
           } else {
             mmi = null;
           }
-          Monitor.gcStatus = fetchGcStatus();
+          gcStatus = fetchGcStatus();
         } catch (Exception e) {
           mmi = null;
           log.info("Error fetching stats: ", e);
@@ -291,8 +259,9 @@ public class Monitor implements HighlyAvailableService {
             MasterClient.close(client);
           }
         }
-        if (mmi == null)
+        if (mmi == null) {
           sleepUninterruptibly(1, TimeUnit.SECONDS);
+        }
       }
       if (mmi != null) {
         int majorCompactions = 0;
@@ -338,24 +307,25 @@ public class Monitor implements HighlyAvailableService {
           totalTabletCount += tInfo.tablets;
           totalTables++;
         }
-        Monitor.totalIngestRate = totalIngestRate;
-        Monitor.totalTables = totalTables;
+        this.totalIngestRate = totalIngestRate;
+        this.totalTables = totalTables;
         totalIngestByteRate = totalIngestByteRate / 1000000.0;
-        Monitor.totalQueryRate = totalQueryRate;
-        Monitor.totalScanRate = totalScanRate;
+        this.totalQueryRate = totalQueryRate;
+        this.totalScanRate = totalScanRate;
         totalQueryByteRate = totalQueryByteRate / 1000000.0;
-        Monitor.totalEntries = totalEntries;
-        Monitor.totalTabletCount = totalTabletCount;
-        Monitor.totalHoldTime = totalHoldTime;
-        Monitor.totalLookups = totalLookups;
+        this.totalEntries = totalEntries;
+        this.totalTabletCount = totalTabletCount;
+        this.totalHoldTime = totalHoldTime;
+        this.totalLookups = totalLookups;
 
         ingestRateOverTime.add(new Pair<>(currentTime, totalIngestRate));
         ingestByteRateOverTime.add(new Pair<>(currentTime, totalIngestByteRate));
 
         double totalLoad = 0.;
         for (TabletServerStatus status : mmi.tServerInfo) {
-          if (status != null)
+          if (status != null) {
             totalLoad += status.osLoad;
+          }
         }
         loadOverTime.add(new Pair<>(currentTime, totalLoad));
 
@@ -375,18 +345,19 @@ public class Monitor implements HighlyAvailableService {
             dataCacheRequestTracker);
       }
       try {
-        Monitor.problemSummary = ProblemReports.getInstance(getContext()).summarize();
-        Monitor.problemException = null;
+        this.problemSummary = ProblemReports.getInstance(context).summarize();
+        this.problemException = null;
       } catch (Exception e) {
         log.info("Failed to obtain problem reports ", e);
-        Monitor.problemSummary = Collections.emptyMap();
-        Monitor.problemException = e;
+        this.problemSummary = Collections.emptyMap();
+        this.problemException = e;
       }
 
     } finally {
-      synchronized (Monitor.class) {
-        fetching = false;
-        lastRecalc.set(currentTime);
+      lastRecalc.set(currentTime);
+      // stop fetching; log an error if this thread wasn't already fetching
+      if (!fetching.compareAndSet(true, false)) {
+        throw new AssertionError("Not supposed to happen; somebody broke this code");
       }
     }
   }
@@ -394,14 +365,15 @@ public class Monitor implements HighlyAvailableService {
   private static void calcCacheHitRate(List<Pair<Long,Double>> hitRate, long currentTime,
       EventCounter cacheHits, EventCounter cacheReq) {
     long req = cacheReq.calculateCount();
-    if (req > 0)
+    if (req > 0) {
       hitRate.add(
           new Pair<>(currentTime, cacheHits.calculateCount() / (double) cacheReq.calculateCount()));
-    else
+    } else {
       hitRate.add(new Pair<>(currentTime, null));
+    }
   }
 
-  private static GCStatus fetchGcStatus() {
+  private GCStatus fetchGcStatus() {
     GCStatus result = null;
     HostAndPort address = null;
     try {
@@ -416,7 +388,7 @@ public class Monitor implements HighlyAvailableService {
         GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(),
             address, context);
         try {
-          result = client.getStatus(TraceUtil.traceInfo(), getContext().rpcCreds());
+          result = client.getStatus(TraceUtil.traceInfo(), context.rpcCreds());
         } finally {
           ThriftUtil.returnClient(client);
         }
@@ -428,33 +400,26 @@ public class Monitor implements HighlyAvailableService {
   }
 
   public static void main(String[] args) {
-
     final String app = "monitor";
     ServerOpts opts = new ServerOpts();
     opts.parseArgs(app, args);
-    Monitor.context = new ServerContext(opts.getSiteConfiguration());
+    ServerContext context = new ServerContext(opts.getSiteConfiguration());
     context.setupServer(app, Monitor.class.getName(), opts.getAddress());
     try {
-      config = context.getServerConfFactory();
-      Monitor monitor = new Monitor();
-      // Servlets need access to limit requests when the monitor is not active, but Servlets are
-      // instantiated via reflection. Expose the service this way instead.
-      Monitor.HA_SERVICE_INSTANCE = monitor;
+      Monitor monitor = new Monitor(context);
       monitor.run();
     } finally {
       context.teardownServer();
     }
   }
 
-  private static long START_TIME;
-
+  @Override
   public void run() {
-    Monitor.START_TIME = System.currentTimeMillis();
-    int[] ports = config.getSystemConfiguration().getPort(Property.MONITOR_PORT);
+    int[] ports = context.getConfiguration().getPort(Property.MONITOR_PORT);
     for (int port : ports) {
       try {
         log.debug("Creating monitor on port {}", port);
-        server = new EmbeddedWebServer(context.getHostname(), port);
+        server = new EmbeddedWebServer(this, port);
         server.addServlet(getDefaultServlet(), "/resources/*");
         server.addServlet(getRestServlet(), "/rest/*");
         server.addServlet(getViewServlet(), "/*");
@@ -505,34 +470,27 @@ public class Monitor implements HighlyAvailableService {
     new Daemon(new LoggingRunnable(log, new ZooKeeperStatus(context)), "ZooKeeperStatus").start();
 
     // need to regularly fetch data so plot data is updated
-    new Daemon(new LoggingRunnable(log, new Runnable() {
-
-      @Override
-      public void run() {
-        while (true) {
-          try {
-            Monitor.fetchData();
-          } catch (Exception e) {
-            log.warn("{}", e.getMessage(), e);
-          }
-
-          sleepUninterruptibly(333, TimeUnit.MILLISECONDS);
+    new Daemon(new LoggingRunnable(log, () -> {
+      while (true) {
+        try {
+          fetchData();
+        } catch (Exception e) {
+          log.warn("{}", e.getMessage(), e);
         }
 
+        sleepUninterruptibly(333, TimeUnit.MILLISECONDS);
       }
+
     }), "Data fetcher").start();
 
-    new Daemon(new LoggingRunnable(log, new Runnable() {
-      @Override
-      public void run() {
-        while (true) {
-          try {
-            Monitor.fetchScans();
-          } catch (Exception e) {
-            log.warn("{}", e.getMessage(), e);
-          }
-          sleepUninterruptibly(5, TimeUnit.SECONDS);
+    new Daemon(new LoggingRunnable(log, () -> {
+      while (true) {
+        try {
+          fetchScans();
+        } catch (Exception e) {
+          log.warn("{}", e.getMessage(), e);
         }
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
       }
     }), "Scan scanner").start();
 
@@ -550,10 +508,32 @@ public class Monitor implements HighlyAvailableService {
     });
   }
 
+  public static class MonitorFactory extends AbstractBinder implements Factory<Monitor> {
+
+    private final Monitor monitor;
+
+    public MonitorFactory(Monitor monitor) {
+      this.monitor = monitor;
+    }
+
+    @Override
+    public Monitor provide() {
+      return monitor;
+    }
+
+    @Override
+    public void dispose(Monitor instance) {}
+
+    @Override
+    protected void configure() {
+      bindFactory(this).to(Monitor.class).in(Singleton.class);
+    }
+  }
+
   private ServletHolder getViewServlet() {
     final ResourceConfig rc = new ResourceConfig().packages("org.apache.accumulo.monitor.view")
-        .register(
-            new LoggingFeature(java.util.logging.Logger.getLogger(this.getClass().getSimpleName())))
+        .register(new MonitorFactory(this))
+        .register(new LoggingFeature(java.util.logging.Logger.getLogger(this.getClass().getName())))
         .register(FreemarkerMvcFeature.class)
         .property(MvcFeature.TEMPLATE_BASE_PATH, "/org/apache/accumulo/monitor/templates");
     return new ServletHolder(new ServletContainer(rc));
@@ -561,8 +541,8 @@ public class Monitor implements HighlyAvailableService {
 
   private ServletHolder getRestServlet() {
     final ResourceConfig rc = new ResourceConfig().packages("org.apache.accumulo.monitor.rest")
-        .register(
-            new LoggingFeature(java.util.logging.Logger.getLogger(this.getClass().getSimpleName())))
+        .register(new MonitorFactory(this))
+        .register(new LoggingFeature(java.util.logging.Logger.getLogger(this.getClass().getName())))
         .register(JacksonFeature.class);
     return new ServletHolder(new ServletContainer(rc));
   }
@@ -583,17 +563,15 @@ public class Monitor implements HighlyAvailableService {
     }
   }
 
-  static final Map<HostAndPort,ScanStats> allScans = new HashMap<>();
+  private final Map<HostAndPort,ScanStats> allScans = new HashMap<>();
 
-  public static Map<HostAndPort,ScanStats> getScans() {
+  public Map<HostAndPort,ScanStats> getScans() {
     synchronized (allScans) {
       return new HashMap<>(allScans);
     }
   }
 
-  public static void fetchScans() throws Exception {
-    if (context == null)
-      return;
+  private void fetchScans() throws Exception {
     for (String server : context.instanceOperations().getTabletServers()) {
       final HostAndPort parsedServer = HostAndPort.fromString(server);
       Client tserver = ThriftUtil.getTServerClient(parsedServer, context);
@@ -672,7 +650,7 @@ public class Monitor implements HighlyAvailableService {
       monitorLock.tryToCancelAsyncLockOrUnlock();
 
       sleepUninterruptibly(
-          getContext().getConfiguration().getTimeInMillis(Property.MONITOR_LOCK_CHECK_INTERVAL),
+          context.getConfiguration().getTimeInMillis(Property.MONITOR_LOCK_CHECK_INTERVAL),
           TimeUnit.MILLISECONDS);
     }
 
@@ -694,12 +672,7 @@ public class Monitor implements HighlyAvailableService {
 
     @Override
     public void unableToMonitorLockNode(final Throwable e) {
-      Halt.halt(-1, new Runnable() {
-        @Override
-        public void run() {
-          log.error("No longer able to monitor Monitor lock node", e);
-        }
-      });
+      Halt.halt(-1, () -> log.error("No longer able to monitor Monitor lock node", e));
 
     }
 
@@ -735,107 +708,107 @@ public class Monitor implements HighlyAvailableService {
     }
   }
 
-  public static MasterMonitorInfo getMmi() {
+  public MasterMonitorInfo getMmi() {
     return mmi;
   }
 
-  public static int getTotalTables() {
+  public int getTotalTables() {
     return totalTables;
   }
 
-  public static int getTotalTabletCount() {
+  public int getTotalTabletCount() {
     return totalTabletCount;
   }
 
-  public static long getTotalEntries() {
+  public long getTotalEntries() {
     return totalEntries;
   }
 
-  public static double getTotalIngestRate() {
+  public double getTotalIngestRate() {
     return totalIngestRate;
   }
 
-  public static double getTotalQueryRate() {
+  public double getTotalQueryRate() {
     return totalQueryRate;
   }
 
-  public static double getTotalScanRate() {
+  public double getTotalScanRate() {
     return totalScanRate;
   }
 
-  public static long getTotalHoldTime() {
+  public long getTotalHoldTime() {
     return totalHoldTime;
   }
 
-  public static Exception getProblemException() {
+  public Exception getProblemException() {
     return problemException;
   }
 
-  public static Map<TableId,Map<ProblemType,Integer>> getProblemSummary() {
+  public Map<TableId,Map<ProblemType,Integer>> getProblemSummary() {
     return problemSummary;
   }
 
-  public static GCStatus getGcStatus() {
+  public GCStatus getGcStatus() {
     return gcStatus;
   }
 
-  public static long getTotalLookups() {
+  public long getTotalLookups() {
     return totalLookups;
   }
 
-  public static long getStartTime() {
+  public long getStartTime() {
     return START_TIME;
   }
 
-  public static List<Pair<Long,Double>> getLoadOverTime() {
+  public List<Pair<Long,Double>> getLoadOverTime() {
     return new ArrayList<>(loadOverTime);
   }
 
-  public static List<Pair<Long,Double>> getIngestRateOverTime() {
+  public List<Pair<Long,Double>> getIngestRateOverTime() {
     return new ArrayList<>(ingestRateOverTime);
   }
 
-  public static List<Pair<Long,Double>> getIngestByteRateOverTime() {
+  public List<Pair<Long,Double>> getIngestByteRateOverTime() {
     return new ArrayList<>(ingestByteRateOverTime);
   }
 
-  public static List<Pair<Long,Integer>> getMinorCompactionsOverTime() {
+  public List<Pair<Long,Integer>> getMinorCompactionsOverTime() {
     return new ArrayList<>(minorCompactionsOverTime);
   }
 
-  public static List<Pair<Long,Integer>> getMajorCompactionsOverTime() {
+  public List<Pair<Long,Integer>> getMajorCompactionsOverTime() {
     return new ArrayList<>(majorCompactionsOverTime);
   }
 
-  public static List<Pair<Long,Double>> getLookupsOverTime() {
+  public List<Pair<Long,Double>> getLookupsOverTime() {
     return new ArrayList<>(lookupsOverTime);
   }
 
-  public static double getLookupRate() {
+  public double getLookupRate() {
     return lookupRateTracker.calculateRate();
   }
 
-  public static List<Pair<Long,Integer>> getQueryRateOverTime() {
+  public List<Pair<Long,Integer>> getQueryRateOverTime() {
     return new ArrayList<>(queryRateOverTime);
   }
 
-  public static List<Pair<Long,Integer>> getScanRateOverTime() {
+  public List<Pair<Long,Integer>> getScanRateOverTime() {
     return new ArrayList<>(scanRateOverTime);
   }
 
-  public static List<Pair<Long,Double>> getQueryByteRateOverTime() {
+  public List<Pair<Long,Double>> getQueryByteRateOverTime() {
     return new ArrayList<>(queryByteRateOverTime);
   }
 
-  public static List<Pair<Long,Double>> getIndexCacheHitRateOverTime() {
+  public List<Pair<Long,Double>> getIndexCacheHitRateOverTime() {
     return new ArrayList<>(indexCacheHitRateOverTime);
   }
 
-  public static List<Pair<Long,Double>> getDataCacheHitRateOverTime() {
+  public List<Pair<Long,Double>> getDataCacheHitRateOverTime() {
     return new ArrayList<>(dataCacheHitRateOverTime);
   }
 
-  public static ServerContext getContext() {
+  public ServerContext getContext() {
     return context;
   }
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
index c8cf7a4..ab440f6 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.monitor.rest;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -39,6 +40,9 @@ import org.apache.accumulo.monitor.rest.tservers.TabletServer;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class XMLResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates summary of the Monitor
    *
@@ -46,18 +50,18 @@ public class XMLResource {
    */
   public SummaryInformation getInformation() {
 
-    MasterMonitorInfo mmi = Monitor.getMmi();
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       throw new WebApplicationException(Status.INTERNAL_SERVER_ERROR);
     }
 
     // Add Monitor information
     SummaryInformation xml = new SummaryInformation(mmi.tServerInfo.size(),
-        MasterResource.getTables(), TablesResource.getTables());
+        MasterResource.getTables(monitor), TablesResource.getTables(monitor));
 
     // Add tserver information
     for (TabletServerStatus status : mmi.tServerInfo) {
-      xml.addTabletServer(new TabletServer(status));
+      xml.addTabletServer(new TabletServer(monitor, status));
     }
 
     return xml;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
index 40c46e9..d337e53 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.monitor.rest.bulkImports;
 
 import java.util.List;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -37,6 +38,9 @@ import org.apache.accumulo.monitor.Monitor;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class BulkImportResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates bulk import and tserver bulk imports with the information from the Monitor
    *
@@ -48,13 +52,13 @@ public class BulkImportResource {
     BulkImport bulkImport = new BulkImport();
 
     // Generating Bulk Import and adding it to the return object
-    for (BulkImportStatus bulk : Monitor.getMmi().bulkImports) {
+    for (BulkImportStatus bulk : monitor.getMmi().bulkImports) {
       bulkImport
           .addBulkImport(new BulkImportInformation(bulk.filename, bulk.startTime, bulk.state));
     }
 
     // Generating TServer Bulk Import and adding it to the return object
-    for (TabletServerStatus tserverInfo : Monitor.getMmi().getTServerInfo()) {
+    for (TabletServerStatus tserverInfo : monitor.getMmi().getTServerInfo()) {
       int size = 0;
       long oldest = 0L;
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/gc/GarbageCollectorResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/gc/GarbageCollectorResource.java
index 5cdc58a..7f67a7d 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/gc/GarbageCollectorResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/gc/GarbageCollectorResource.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.monitor.rest.gc;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -33,6 +34,9 @@ import org.apache.accumulo.monitor.Monitor;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class GarbageCollectorResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Returns the garbage collector status
    *
@@ -40,7 +44,7 @@ public class GarbageCollectorResource {
    */
   @GET
   public GarbageCollectorStatus getStatus() {
-    return new GarbageCollectorStatus(Monitor.getGcStatus());
+    return new GarbageCollectorStatus(monitor.getGcStatus());
   }
 
   /**
@@ -51,7 +55,7 @@ public class GarbageCollectorResource {
   @Path("files")
   @GET
   public GarbageCollection getFileStatus() {
-    GCStatus gcStatus = Monitor.getGcStatus();
+    GCStatus gcStatus = monitor.getGcStatus();
     if (gcStatus == null) {
       return GarbageCollection.getEmpty();
     }
@@ -66,7 +70,7 @@ public class GarbageCollectorResource {
   @Path("files/last")
   @GET
   public GarbageCollectorCycle getLastCycle() {
-    GCStatus status = Monitor.getGcStatus();
+    GCStatus status = monitor.getGcStatus();
     if (status == null) {
       return GarbageCollectorCycle.getEmpty();
     }
@@ -81,7 +85,7 @@ public class GarbageCollectorResource {
   @Path("files/current")
   @GET
   public GarbageCollectorCycle getCurrentCycle() {
-    GCStatus status = Monitor.getGcStatus();
+    GCStatus status = monitor.getGcStatus();
     if (status == null) {
       return GarbageCollectorCycle.getEmpty();
     }
@@ -96,7 +100,7 @@ public class GarbageCollectorResource {
   @Path("wals")
   @GET
   public GarbageCollection getWalStatus() {
-    GCStatus gcStatus = Monitor.getGcStatus();
+    GCStatus gcStatus = monitor.getGcStatus();
     if (gcStatus == null) {
       return GarbageCollection.getEmpty();
     }
@@ -111,7 +115,7 @@ public class GarbageCollectorResource {
   @Path("wals/last")
   @GET
   public GarbageCollectorCycle getLastWalCycle() {
-    GCStatus status = Monitor.getGcStatus();
+    GCStatus status = monitor.getGcStatus();
     if (status == null) {
       return GarbageCollectorCycle.getEmpty();
     }
@@ -126,7 +130,7 @@ public class GarbageCollectorResource {
   @Path("wals/current")
   @GET
   public GarbageCollectorCycle getCurrentWalCycle() {
-    GCStatus status = Monitor.getGcStatus();
+    GCStatus status = monitor.getGcStatus();
     if (status == null) {
       return GarbageCollectorCycle.getEmpty();
     }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
index 16322a4..a8bad25 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -53,12 +54,8 @@ import org.apache.accumulo.server.master.state.TabletServerState;
 public class MasterResource {
   public static final String NO_MASTERS = "No Masters running";
 
-  /**
-   * Gets the MasterMonitorInfo, allowing for mocking frameworks for testability
-   */
-  protected static MasterMonitorInfo getMmi() {
-    return Monitor.getMmi();
-  }
+  @Inject
+  private Monitor monitor;
 
   /**
    * Generates a master information JSON object
@@ -66,13 +63,16 @@ public class MasterResource {
    * @return master JSON object
    */
   @GET
-  public static MasterInformation getTables() {
+  public MasterInformation getTables() {
+    return getTables(monitor);
+  }
 
+  public static MasterInformation getTables(Monitor monitor) {
     MasterInformation masterInformation;
-    MasterMonitorInfo mmi = Monitor.getMmi();
+    MasterMonitorInfo mmi = monitor.getMmi();
 
     if (mmi != null) {
-      GCStatus gcStatusObj = Monitor.getGcStatus();
+      GCStatus gcStatusObj = monitor.getGcStatus();
       String gcStatus = "Waiting";
       String label = "";
       if (gcStatusObj != null) {
@@ -97,31 +97,31 @@ public class MasterResource {
       for (DeadServer down : mmi.deadTabletServers) {
         tservers.add(down.server);
       }
-      List<String> masters = Monitor.getContext().getMasterLocations();
+      List<String> masters = monitor.getContext().getMasterLocations();
 
       String master = masters.size() == 0 ? "Down"
           : AddressUtil.parseAddress(masters.get(0), false).getHost();
       int onlineTabletServers = mmi.tServerInfo.size();
       int totalTabletServers = tservers.size();
-      int tablets = Monitor.getTotalTabletCount();
+      int tablets = monitor.getTotalTabletCount();
       int unassignedTablets = mmi.unassignedTablets;
-      long entries = Monitor.getTotalEntries();
-      double ingest = Monitor.getTotalIngestRate();
-      double entriesRead = Monitor.getTotalScanRate();
-      double entriesReturned = Monitor.getTotalQueryRate();
-      long holdTime = Monitor.getTotalHoldTime();
+      long entries = monitor.getTotalEntries();
+      double ingest = monitor.getTotalIngestRate();
+      double entriesRead = monitor.getTotalScanRate();
+      double entriesReturned = monitor.getTotalQueryRate();
+      long holdTime = monitor.getTotalHoldTime();
       double osLoad = ManagementFactory.getOperatingSystemMXBean().getSystemLoadAverage();
 
-      int tables = Monitor.getTotalTables();
+      int tables = monitor.getTotalTables();
       int deadTabletServers = mmi.deadTabletServers.size();
-      long lookups = Monitor.getTotalLookups();
-      long uptime = System.currentTimeMillis() - Monitor.getStartTime();
+      long lookups = monitor.getTotalLookups();
+      long uptime = System.currentTimeMillis() - monitor.getStartTime();
 
       masterInformation = new MasterInformation(master, onlineTabletServers, totalTabletServers,
           gcStatus, tablets, unassignedTablets, entries, ingest, entriesRead, entriesReturned,
-          holdTime, osLoad, tables, deadTabletServers, lookups, uptime, label, getGoalState(),
-          getState(), getNumBadTservers(), getServersShuttingDown(), getDeadTservers(),
-          getDeadLoggers());
+          holdTime, osLoad, tables, deadTabletServers, lookups, uptime, label,
+          getGoalState(monitor), getState(monitor), getNumBadTservers(monitor),
+          getServersShuttingDown(monitor), getDeadTservers(monitor), getDeadLoggers(monitor));
     } else {
       masterInformation = new MasterInformation();
     }
@@ -133,8 +133,8 @@ public class MasterResource {
    *
    * @return master state
    */
-  public static String getState() {
-    MasterMonitorInfo mmi = getMmi();
+  public static String getState(Monitor monitor) {
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       return NO_MASTERS;
     }
@@ -146,8 +146,8 @@ public class MasterResource {
    *
    * @return master goal state
    */
-  public static String getGoalState() {
-    MasterMonitorInfo mmi = getMmi();
+  public static String getGoalState(Monitor monitor) {
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       return NO_MASTERS;
     }
@@ -159,8 +159,8 @@ public class MasterResource {
    *
    * @return dead server list
    */
-  public static DeadServerList getDeadTservers() {
-    MasterMonitorInfo mmi = getMmi();
+  public static DeadServerList getDeadTservers(Monitor monitor) {
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       return new DeadServerList();
     }
@@ -179,8 +179,8 @@ public class MasterResource {
    *
    * @return dead logger list
    */
-  public static DeadLoggerList getDeadLoggers() {
-    MasterMonitorInfo mmi = getMmi();
+  public static DeadLoggerList getDeadLoggers(Monitor monitor) {
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       return new DeadLoggerList();
     }
@@ -199,8 +199,8 @@ public class MasterResource {
    *
    * @return bad tserver list
    */
-  public static BadTabletServers getNumBadTservers() {
-    MasterMonitorInfo mmi = getMmi();
+  public static BadTabletServers getNumBadTservers(Monitor monitor) {
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       return new BadTabletServers();
     }
@@ -231,10 +231,10 @@ public class MasterResource {
    *
    * @return servers shutting down list
    */
-  public static ServersShuttingDown getServersShuttingDown() {
+  public static ServersShuttingDown getServersShuttingDown(Monitor monitor) {
     ServersShuttingDown servers = new ServersShuttingDown();
     // Add new servers to the list
-    for (String server : Monitor.getMmi().serversShuttingDown) {
+    for (String server : monitor.getMmi().serversShuttingDown) {
       servers.addServerShuttingDown(new ServerShuttingDownInformation(server));
     }
     return servers;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
index f79ad80..f3e5da3 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import javax.inject.Inject;
 import javax.validation.constraints.NotNull;
 import javax.validation.constraints.Pattern;
 import javax.ws.rs.Consumes;
@@ -54,6 +55,9 @@ import org.slf4j.LoggerFactory;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class ProblemsResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates a list with the problem summary
    *
@@ -65,8 +69,8 @@ public class ProblemsResource {
 
     ProblemSummary problems = new ProblemSummary();
 
-    if (Monitor.getProblemException() == null) {
-      for (Entry<TableId,Map<ProblemType,Integer>> entry : Monitor.getProblemSummary().entrySet()) {
+    if (monitor.getProblemException() == null) {
+      for (Entry<TableId,Map<ProblemType,Integer>> entry : monitor.getProblemSummary().entrySet()) {
         Integer readCount = null, writeCount = null, loadCount = null;
 
         for (ProblemType pt : ProblemType.values()) {
@@ -80,7 +84,7 @@ public class ProblemsResource {
           }
         }
 
-        String tableName = Tables.getPrintableTableInfoFromId(Monitor.getContext(), entry.getKey());
+        String tableName = Tables.getPrintableTableInfoFromId(monitor.getContext(), entry.getKey());
 
         problems.addProblemSummary(new ProblemSummaryInformation(tableName, entry.getKey(),
             readCount, writeCount, loadCount));
@@ -102,7 +106,7 @@ public class ProblemsResource {
       @QueryParam("s") @NotNull @Pattern(regexp = ALPHA_NUM_REGEX_TABLE_ID) String tableID) {
     Logger log = LoggerFactory.getLogger(Monitor.class);
     try {
-      ProblemReports.getInstance(Monitor.getContext()).deleteProblemReports(TableId.of(tableID));
+      ProblemReports.getInstance(monitor.getContext()).deleteProblemReports(TableId.of(tableID));
     } catch (Exception e) {
       log.error("Failed to delete problem reports for table "
           + (StringUtils.isEmpty(tableID) ? StringUtils.EMPTY : sanitize(tableID)), e);
@@ -120,16 +124,17 @@ public class ProblemsResource {
 
     ProblemDetail problems = new ProblemDetail();
 
-    if (Monitor.getProblemException() == null) {
-      for (Entry<TableId,Map<ProblemType,Integer>> entry : Monitor.getProblemSummary().entrySet()) {
+    if (monitor.getProblemException() == null) {
+      for (Entry<TableId,Map<ProblemType,Integer>> entry : monitor.getProblemSummary().entrySet()) {
         ArrayList<ProblemReport> problemReports = new ArrayList<>();
         Iterator<ProblemReport> iter = entry.getKey() == null
-            ? ProblemReports.getInstance(Monitor.getContext()).iterator()
-            : ProblemReports.getInstance(Monitor.getContext()).iterator(entry.getKey());
-        while (iter.hasNext())
+            ? ProblemReports.getInstance(monitor.getContext()).iterator()
+            : ProblemReports.getInstance(monitor.getContext()).iterator(entry.getKey());
+        while (iter.hasNext()) {
           problemReports.add(iter.next());
+        }
         for (ProblemReport pr : problemReports) {
-          String tableName = Tables.getPrintableTableInfoFromId(Monitor.getContext(),
+          String tableName = Tables.getPrintableTableInfoFromId(monitor.getContext(),
               pr.getTableId());
 
           problems.addProblemDetail(
@@ -160,7 +165,7 @@ public class ProblemsResource {
       @QueryParam("ptype") @NotNull @Pattern(regexp = PROBLEM_TYPE_REGEX) String ptype) {
     Logger log = LoggerFactory.getLogger(Monitor.class);
     try {
-      ProblemReports.getInstance(Monitor.getContext()).deleteProblemReport(TableId.of(tableID),
+      ProblemReports.getInstance(monitor.getContext()).deleteProblemReport(TableId.of(tableID),
           ProblemType.valueOf(ptype), resource);
     } catch (Exception e) {
       log.error("Failed to delete problem reports for table "
@@ -179,7 +184,7 @@ public class ProblemsResource {
   @GET
   @Path("exception")
   public Exception getException() {
-    return Monitor.getProblemException();
+    return monitor.getProblemException();
   }
 
 }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java
index 97656a2..6af40cd 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -66,6 +67,9 @@ import org.slf4j.LoggerFactory;
 public class ReplicationResource {
   private static final Logger log = LoggerFactory.getLogger(ReplicationResource.class);
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates the replication table as a JSON object
    *
@@ -74,7 +78,7 @@ public class ReplicationResource {
   @GET
   public List<ReplicationInformation> getReplicationInformation()
       throws AccumuloException, AccumuloSecurityException {
-    final AccumuloClient client = Monitor.getContext();
+    final AccumuloClient client = monitor.getContext();
 
     final TableOperations tops = client.tableOperations();
 
@@ -93,7 +97,7 @@ public class ReplicationResource {
         String peerName = property.getKey().substring(definedPeersPrefix.length());
         ReplicaSystem replica;
         try {
-          replica = replicaSystemFactory.get(Monitor.getContext(), property.getValue());
+          replica = replicaSystemFactory.get(monitor.getContext(), property.getValue());
         } catch (Exception e) {
           log.warn("Could not instantiate ReplicaSystem for {} with configuration {}",
               property.getKey(), property.getValue(), e);
@@ -112,7 +116,7 @@ public class ReplicationResource {
     // Number of files per target we have to replicate
     Map<ReplicationTarget,Long> targetCounts = new HashMap<>();
 
-    Map<String,TableId> tableNameToId = Tables.getNameToIdMap(Monitor.getContext());
+    Map<String,TableId> tableNameToId = Tables.getNameToIdMap(monitor.getContext());
     Map<TableId,String> tableIdToName = invert(tableNameToId);
 
     for (String table : tops.list()) {
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
index 57a9ca0..7b211d5 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.monitor.rest.scans;
 
 import java.util.Map;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -37,6 +38,9 @@ import org.apache.accumulo.monitor.Monitor.ScanStats;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class ScansResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates a new JSON object with scan information
    *
@@ -47,10 +51,10 @@ public class ScansResource {
 
     Scans scans = new Scans();
 
-    Map<HostAndPort,ScanStats> entry = Monitor.getScans();
+    Map<HostAndPort,ScanStats> entry = monitor.getScans();
 
     // Adds new scans to the array
-    for (TabletServerStatus tserverInfo : Monitor.getMmi().getTServerInfo()) {
+    for (TabletServerStatus tserverInfo : monitor.getMmi().getTServerInfo()) {
       ScanStats stats = entry.get(HostAndPort.fromString(tserverInfo.name));
       if (stats != null) {
         scans.addScan(new ScanInformation(tserverInfo, stats.scanCount, stats.oldestScan));
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/statistics/StatisticsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/statistics/StatisticsResource.java
index bdbbc3f..0b2addd 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/statistics/StatisticsResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/statistics/StatisticsResource.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.monitor.rest.statistics;
 import java.util.ArrayList;
 import java.util.List;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -37,6 +38,9 @@ import org.apache.accumulo.monitor.Monitor;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class StatisticsResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates the total lookup rate
    *
@@ -45,7 +49,7 @@ public class StatisticsResource {
   @GET
   @Path("lookupRate")
   public double getLookupRate() {
-    return Monitor.getLookupRate();
+    return monitor.getLookupRate();
   }
 
   /**
@@ -56,7 +60,7 @@ public class StatisticsResource {
   @GET
   @Path("totalTables")
   public int getTotalTables() {
-    return Monitor.getTotalTables();
+    return monitor.getTotalTables();
   }
 
   /**
@@ -67,7 +71,7 @@ public class StatisticsResource {
   @GET
   @Path("totalTabletCount")
   public int getTotalTabletCount() {
-    return Monitor.getTotalTabletCount();
+    return monitor.getTotalTabletCount();
   }
 
   /**
@@ -78,7 +82,7 @@ public class StatisticsResource {
   @GET
   @Path("totalEntries")
   public long getTotalEntries() {
-    return Monitor.getTotalEntries();
+    return monitor.getTotalEntries();
   }
 
   /**
@@ -89,7 +93,7 @@ public class StatisticsResource {
   @GET
   @Path("totalIngestRate")
   public double getTotalIngestRate() {
-    return Monitor.getTotalIngestRate();
+    return monitor.getTotalIngestRate();
   }
 
   /**
@@ -100,7 +104,7 @@ public class StatisticsResource {
   @GET
   @Path("totalQueryRate")
   public double getTotalQueryRate() {
-    return Monitor.getTotalQueryRate();
+    return monitor.getTotalQueryRate();
   }
 
   /**
@@ -111,7 +115,7 @@ public class StatisticsResource {
   @GET
   @Path("totalScanRate")
   public double getTotalScanRate() {
-    return Monitor.getTotalScanRate();
+    return monitor.getTotalScanRate();
   }
 
   /**
@@ -122,7 +126,7 @@ public class StatisticsResource {
   @GET
   @Path("totalHoldTime")
   public long getTotalHoldTime() {
-    return Monitor.getTotalHoldTime();
+    return monitor.getTotalHoldTime();
   }
 
   /**
@@ -133,7 +137,7 @@ public class StatisticsResource {
   @GET
   @Path("gcStatus")
   public GCStatus getGcStatus() {
-    return Monitor.getGcStatus();
+    return monitor.getGcStatus();
   }
 
   /**
@@ -144,7 +148,7 @@ public class StatisticsResource {
   @GET
   @Path("totalLookups")
   public long getTotalLookups() {
-    return Monitor.getTotalLookups();
+    return monitor.getTotalLookups();
   }
 
   /**
@@ -155,7 +159,7 @@ public class StatisticsResource {
   @GET
   @Path("time/scanRate")
   public List<Pair<Long,Integer>> getScanRate() {
-    return Monitor.getScanRateOverTime();
+    return monitor.getScanRateOverTime();
   }
 
   /**
@@ -166,7 +170,7 @@ public class StatisticsResource {
   @GET
   @Path("time/queryRate")
   public List<Pair<Long,Integer>> getQueryRate() {
-    return Monitor.getQueryRateOverTime();
+    return monitor.getQueryRateOverTime();
   }
 
   /**
@@ -180,9 +184,9 @@ public class StatisticsResource {
 
     List<Pair<String,List<Pair<Long,Integer>>>> scanEntries = new ArrayList<>();
 
-    Pair<String,List<Pair<Long,Integer>>> read = new Pair<>("Read", Monitor.getScanRateOverTime());
+    Pair<String,List<Pair<Long,Integer>>> read = new Pair<>("Read", monitor.getScanRateOverTime());
     Pair<String,List<Pair<Long,Integer>>> returned = new Pair<>("Returned",
-        Monitor.getQueryRateOverTime());
+        monitor.getQueryRateOverTime());
 
     scanEntries.add(read);
     scanEntries.add(returned);
@@ -198,7 +202,7 @@ public class StatisticsResource {
   @GET
   @Path("time/queryByteRate")
   public List<Pair<Long,Double>> getQueryByteRate() {
-    return Monitor.getQueryByteRateOverTime();
+    return monitor.getQueryByteRateOverTime();
   }
 
   /**
@@ -209,7 +213,7 @@ public class StatisticsResource {
   @GET
   @Path("time/load")
   public List<Pair<Long,Double>> getLoad() {
-    return Monitor.getLoadOverTime();
+    return monitor.getLoadOverTime();
   }
 
   /**
@@ -220,7 +224,7 @@ public class StatisticsResource {
   @GET
   @Path("time/ingestRate")
   public List<Pair<Long,Double>> getIngestRate() {
-    return Monitor.getIngestRateOverTime();
+    return monitor.getIngestRateOverTime();
   }
 
   /**
@@ -231,7 +235,7 @@ public class StatisticsResource {
   @GET
   @Path("time/ingestByteRate")
   public List<Pair<Long,Double>> getIngestByteRate() {
-    return Monitor.getIngestByteRateOverTime();
+    return monitor.getIngestByteRateOverTime();
   }
 
   /**
@@ -242,7 +246,7 @@ public class StatisticsResource {
   @GET
   @Path("time/minorCompactions")
   public List<Pair<Long,Integer>> getMinorCompactions() {
-    return Monitor.getMinorCompactionsOverTime();
+    return monitor.getMinorCompactionsOverTime();
   }
 
   /**
@@ -253,7 +257,7 @@ public class StatisticsResource {
   @GET
   @Path("time/majorCompactions")
   public List<Pair<Long,Integer>> getMajorCompactions() {
-    return Monitor.getMajorCompactionsOverTime();
+    return monitor.getMajorCompactionsOverTime();
   }
 
   /**
@@ -264,7 +268,7 @@ public class StatisticsResource {
   @GET
   @Path("time/lookups")
   public List<Pair<Long,Double>> getLookups() {
-    return Monitor.getLookupsOverTime();
+    return monitor.getLookupsOverTime();
   }
 
   /**
@@ -275,7 +279,7 @@ public class StatisticsResource {
   @GET
   @Path("time/indexCacheHitRate")
   public List<Pair<Long,Double>> getIndexCacheHitRate() {
-    return Monitor.getIndexCacheHitRateOverTime();
+    return monitor.getIndexCacheHitRateOverTime();
   }
 
   /**
@@ -286,6 +290,6 @@ public class StatisticsResource {
   @GET
   @Path("time/dataCacheHitRate")
   public List<Pair<Long,Double>> getDataCacheHitRate() {
-    return Monitor.getDataCacheHitRateOverTime();
+    return monitor.getDataCacheHitRateOverTime();
   }
 }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
index b8d039d..c89b370 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.monitor.rest.status;
 
 import java.util.List;
 
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -37,6 +38,9 @@ import org.apache.log4j.Level;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class StatusResource {
 
+  @Inject
+  private Monitor monitor;
+
   public enum Status {
     OK, ERROR, WARN
   }
@@ -54,19 +58,19 @@ public class StatusResource {
     Status gcStatus;
     Status tServerStatus = Status.ERROR;
 
-    if (Monitor.getMmi() != null) {
-      if (Monitor.getGcStatus() != null) {
+    if (monitor.getMmi() != null) {
+      if (monitor.getGcStatus() != null) {
         gcStatus = Status.OK;
       } else {
         gcStatus = Status.ERROR;
       }
 
-      List<String> masters = Monitor.getContext().getMasterLocations();
+      List<String> masters = monitor.getContext().getMasterLocations();
       masterStatus = masters.size() == 0 ? Status.ERROR : Status.OK;
 
-      int tServerUp = Monitor.getMmi().getTServerInfoSize();
-      int tServerDown = Monitor.getMmi().getDeadTabletServersSize();
-      int tServerBad = Monitor.getMmi().getBadTServersSize();
+      int tServerUp = monitor.getMmi().getTServerInfoSize();
+      int tServerDown = monitor.getMmi().getDeadTabletServersSize();
+      int tServerBad = monitor.getMmi().getBadTServersSize();
 
       /*
        * If there are no dead or bad servers and there are tservers up, status is OK, if there are
@@ -82,7 +86,7 @@ public class StatusResource {
       }
     } else {
       masterStatus = Status.ERROR;
-      if (Monitor.getGcStatus() == null) {
+      if (monitor.getGcStatus() == null) {
         gcStatus = Status.ERROR;
       } else {
         gcStatus = Status.OK;
@@ -99,7 +103,7 @@ public class StatusResource {
       }
     }
 
-    int numProblems = Monitor.getProblemSummary().entrySet().size();
+    int numProblems = monitor.getProblemSummary().entrySet().size();
 
     status = new StatusInformation(masterStatus.toString(), gcStatus.toString(),
         tServerStatus.toString(), logs.size(), logsHaveError, numProblems);
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
index 83df50f..3cd8007 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
@@ -25,6 +25,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import javax.inject.Inject;
 import javax.validation.constraints.NotNull;
 import javax.validation.constraints.Pattern;
 import javax.ws.rs.GET;
@@ -61,6 +62,9 @@ import org.apache.hadoop.io.Text;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class TablesResource {
 
+  @Inject
+  private Monitor monitor;
+
   private static final TabletServerStatus NO_STATUS = new TabletServerStatus();
 
   /**
@@ -69,20 +73,25 @@ public class TablesResource {
    * @return list with all tables
    */
   @GET
-  public static TableInformationList getTables() {
+  public TableInformationList getTables() {
+    return getTables(monitor);
+  }
 
+  public static TableInformationList getTables(Monitor monitor) {
     TableInformationList tableList = new TableInformationList();
     SortedMap<TableId,TableInfo> tableStats = new TreeMap<>();
 
-    if (Monitor.getMmi() != null && Monitor.getMmi().tableMap != null)
-      for (Map.Entry<String,TableInfo> te : Monitor.getMmi().tableMap.entrySet())
+    if (monitor.getMmi() != null && monitor.getMmi().tableMap != null) {
+      for (Map.Entry<String,TableInfo> te : monitor.getMmi().tableMap.entrySet()) {
         tableStats.put(TableId.of(te.getKey()), te.getValue());
+      }
+    }
 
-    Map<String,Double> compactingByTable = TableInfoUtil.summarizeTableStats(Monitor.getMmi());
-    TableManager tableManager = Monitor.getContext().getTableManager();
+    Map<String,Double> compactingByTable = TableInfoUtil.summarizeTableStats(monitor.getMmi());
+    TableManager tableManager = monitor.getContext().getTableManager();
 
     // Add tables to the list
-    for (Map.Entry<String,TableId> entry : Tables.getNameToIdMap(Monitor.getContext()).entrySet()) {
+    for (Map.Entry<String,TableId> entry : Tables.getNameToIdMap(monitor.getContext()).entrySet()) {
       String tableName = entry.getKey();
       TableId tableId = entry.getValue();
       TableInfo tableInfo = tableStats.get(tableId);
@@ -90,8 +99,9 @@ public class TablesResource {
 
       if (tableInfo != null && !tableState.equals(TableState.OFFLINE)) {
         Double holdTime = compactingByTable.get(tableId.canonical());
-        if (holdTime == null)
+        if (holdTime == null) {
           holdTime = 0.;
+        }
 
         tableList.addTable(
             new TableInformation(tableName, tableId, tableInfo, holdTime, tableState.name()));
@@ -113,10 +123,10 @@ public class TablesResource {
   @GET
   public TabletServers getParticipatingTabletServers(@PathParam("tableId") @NotNull @Pattern(
       regexp = ALPHA_NUM_REGEX_TABLE_ID) String tableIdStr) {
-    String rootTabletLocation = Monitor.getContext().getRootTabletLocation();
+    String rootTabletLocation = monitor.getContext().getRootTabletLocation();
     TableId tableId = TableId.of(tableIdStr);
 
-    TabletServers tabletServers = new TabletServers(Monitor.getMmi().tServerInfo.size());
+    TabletServers tabletServers = new TabletServers(monitor.getMmi().tServerInfo.size());
 
     if (StringUtils.isBlank(tableIdStr)) {
       return tabletServers;
@@ -128,7 +138,7 @@ public class TablesResource {
     } else {
       String systemTableName = MetadataTable.ID.equals(tableId) ? RootTable.NAME
           : MetadataTable.NAME;
-      MetaDataTableScanner scanner = new MetaDataTableScanner(Monitor.getContext(),
+      MetaDataTableScanner scanner = new MetaDataTableScanner(monitor.getContext(),
           new Range(TabletsSection.getRow(tableId, new Text()),
               TabletsSection.getRow(tableId, null)),
           systemTableName);
@@ -148,11 +158,12 @@ public class TablesResource {
     }
 
     List<TabletServerStatus> tservers = new ArrayList<>();
-    if (Monitor.getMmi() != null) {
-      for (TabletServerStatus tss : Monitor.getMmi().tServerInfo) {
+    if (monitor.getMmi() != null) {
+      for (TabletServerStatus tss : monitor.getMmi().tServerInfo) {
         try {
-          if (tss.name != null && locs.contains(tss.name))
+          if (tss.name != null && locs.contains(tss.name)) {
             tservers.add(tss);
+          }
         } catch (Exception ex) {
           return tabletServers;
         }
@@ -161,16 +172,19 @@ public class TablesResource {
 
     // Adds tservers to the list
     for (TabletServerStatus status : tservers) {
-      if (status == null)
+      if (status == null) {
         status = NO_STATUS;
+      }
       TableInfo summary = TableInfoUtil.summarizeTableStats(status);
-      if (tableId != null)
+      if (tableId != null) {
         summary = status.tableMap.get(tableId.canonical());
-      if (summary == null)
+      }
+      if (summary == null) {
         continue;
+      }
 
       TabletServer tabletServerInfo = new TabletServer();
-      tabletServerInfo.updateTabletServerInfo(status, summary);
+      tabletServerInfo.server.updateTabletServerInfo(monitor, status, summary);
 
       tabletServers.addTablet(tabletServerInfo);
     }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/TracesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/TracesResource.java
index 80bc8d0..4d985de 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/TracesResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/TracesResource.java
@@ -29,6 +29,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
+import javax.inject.Inject;
 import javax.validation.constraints.Max;
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotNull;
@@ -76,6 +77,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class TracesResource {
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Generates a trace summary
    *
@@ -295,7 +299,7 @@ public class TracesResource {
   }
 
   protected Pair<AccumuloClient,UserGroupInformation> getClient() {
-    AccumuloConfiguration conf = Monitor.getContext().getConfiguration();
+    AccumuloConfiguration conf = monitor.getContext().getConfiguration();
     final boolean saslEnabled = conf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED);
     UserGroupInformation traceUgi = null;
     final String principal;
@@ -339,7 +343,7 @@ public class TracesResource {
       at = null;
     }
 
-    java.util.Properties props = Monitor.getContext().getProperties();
+    java.util.Properties props = monitor.getContext().getProperties();
     AccumuloClient client;
     if (traceUgi != null) {
       try {
@@ -362,7 +366,7 @@ public class TracesResource {
 
   private Scanner getScanner(AccumuloClient client) throws AccumuloException {
     try {
-      AccumuloConfiguration conf = Monitor.getContext().getConfiguration();
+      AccumuloConfiguration conf = monitor.getContext().getConfiguration();
       final String table = conf.get(Property.TRACE_TABLE);
       if (!client.tableOperations().exists(table)) {
         return null;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
index 585b326..0a71765 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
@@ -16,8 +16,8 @@
  */
 package org.apache.accumulo.monitor.rest.tservers;
 
-import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.monitor.Monitor;
 
 /**
  * To use for XML Resource
@@ -27,19 +27,18 @@ import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 public class TabletServer {
 
   // Variable names become JSON keys
-  public TabletServerInformation server = new TabletServerInformation();
+  public final TabletServerInformation server;
 
-  public TabletServer() {}
+  public TabletServer() {
+    server = new TabletServerInformation();
+  }
 
   public TabletServer(TabletServerInformation server) {
     this.server = server;
   }
 
-  public TabletServer(TabletServerStatus status) {
-    server = new TabletServerInformation(status);
+  public TabletServer(Monitor monitor, TabletServerStatus status) {
+    server = new TabletServerInformation(monitor, status);
   }
 
-  public void updateTabletServerInfo(TabletServerStatus status, TableInfo summary) {
-    server.updateTabletServerInfo(status, summary);
-  }
 }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
index 7214463..ee72a03 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
@@ -95,9 +95,9 @@ public class TabletServerInformation {
    * @param thriftStatus
    *          Thrift status to obtain information
    */
-  public TabletServerInformation(TabletServerStatus thriftStatus) {
+  public TabletServerInformation(Monitor monitor, TabletServerStatus thriftStatus) {
     TableInfo summary = TableInfoUtil.summarizeTableStats(thriftStatus);
-    updateTabletServerInfo(thriftStatus, summary);
+    updateTabletServerInfo(monitor, thriftStatus, summary);
   }
 
   /**
@@ -108,7 +108,8 @@ public class TabletServerInformation {
    * @param summary
    *          Table info summary
    */
-  public void updateTabletServerInfo(TabletServerStatus thriftStatus, TableInfo summary) {
+  public void updateTabletServerInfo(Monitor monitor, TabletServerStatus thriftStatus,
+      TableInfo summary) {
 
     long now = System.currentTimeMillis();
 
@@ -159,7 +160,7 @@ public class TabletServerInformation {
     this.ingestMB = cleanNumber(summary.ingestByteRate);
     this.queryMB = cleanNumber(summary.queryByteRate);
 
-    this.scansessions = Monitor.getLookupRate();
+    this.scansessions = monitor.getLookupRate();
     this.scanssessions = this.scansessions; // For backwards compatibility
 
     this.logRecoveries = new ArrayList<>(thriftStatus.logSorts.size());
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
index 11f1742..42885a9 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Base64;
 import java.util.List;
 
+import javax.inject.Inject;
 import javax.validation.constraints.NotNull;
 import javax.validation.constraints.Pattern;
 import javax.ws.rs.Consumes;
@@ -66,6 +67,9 @@ import org.apache.accumulo.server.util.ActionStatsUpdator;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class TabletServerResource {
 
+  @Inject
+  private Monitor monitor;
+
   // Variable names become JSON keys
   private TabletStats total;
   private TabletStats historical;
@@ -77,17 +81,17 @@ public class TabletServerResource {
    */
   @GET
   public TabletServers getTserverSummary() {
-    MasterMonitorInfo mmi = Monitor.getMmi();
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       throw new WebApplicationException(Status.INTERNAL_SERVER_ERROR);
     }
 
     TabletServers tserverInfo = new TabletServers(mmi.tServerInfo.size());
     for (TabletServerStatus status : mmi.tServerInfo) {
-      tserverInfo.addTablet(new TabletServer(status));
+      tserverInfo.addTablet(new TabletServer(monitor, status));
     }
 
-    tserverInfo.addBadTabletServer(MasterResource.getTables());
+    tserverInfo.addBadTabletServer(MasterResource.getTables(monitor));
 
     return tserverInfo;
   }
@@ -102,8 +106,8 @@ public class TabletServerResource {
   @Consumes(MediaType.TEXT_PLAIN)
   public void clearDeadServer(
       @QueryParam("server") @NotNull @Pattern(regexp = HOSTNAME_PORT_REGEX) String server) {
-    DeadServerList obit = new DeadServerList(Monitor.getContext(),
-        Monitor.getContext().getZooKeeperRoot() + Constants.ZDEADTSERVERS);
+    DeadServerList obit = new DeadServerList(monitor.getContext(),
+        monitor.getContext().getZooKeeperRoot() + Constants.ZDEADTSERVERS);
     obit.delete(server);
   }
 
@@ -117,7 +121,7 @@ public class TabletServerResource {
   public TabletServersRecovery getTserverRecovery() {
     TabletServersRecovery recoveryList = new TabletServersRecovery();
 
-    MasterMonitorInfo mmi = Monitor.getMmi();
+    MasterMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
       throw new WebApplicationException(Status.INTERNAL_SERVER_ERROR);
     }
@@ -152,7 +156,7 @@ public class TabletServerResource {
       throws Exception {
 
     boolean tserverExists = false;
-    for (TabletServerStatus ts : Monitor.getMmi().getTServerInfo()) {
+    for (TabletServerStatus ts : monitor.getMmi().getTServerInfo()) {
       if (tserverAddress.equals(ts.getName())) {
         tserverExists = true;
         break;
@@ -180,11 +184,11 @@ public class TabletServerResource {
     List<TabletStats> tsStats = new ArrayList<>();
 
     try {
-      ClientContext context = Monitor.getContext();
+      ClientContext context = monitor.getContext();
       TabletClientService.Client client = ThriftUtil
           .getClient(new TabletClientService.Client.Factory(), address, context);
       try {
-        for (String tableId : Monitor.getMmi().tableMap.keySet()) {
+        for (String tableId : monitor.getMmi().tableMap.keySet()) {
           tsStats.addAll(client.getTabletStats(TraceUtil.traceInfo(), context.rpcCreds(), tableId));
         }
         historical = client.getHistoricalStats(TraceUtil.traceInfo(), context.rpcCreds());
@@ -197,15 +201,19 @@ public class TabletServerResource {
 
     List<CurrentOperations> currentOps = doCurrentOperations(tsStats);
 
-    if (total.minors.num != 0)
+    if (total.minors.num != 0) {
       currentMinorAvg = (long) (total.minors.elapsed / total.minors.num);
-    if (total.minors.elapsed != 0 && total.minors.num != 0)
+    }
+    if (total.minors.elapsed != 0 && total.minors.num != 0) {
       currentMinorStdDev = stddev(total.minors.elapsed, total.minors.num, total.minors.sumDev);
-    if (total.majors.num != 0)
+    }
+    if (total.majors.num != 0) {
       currentMajorAvg = total.majors.elapsed / total.majors.num;
+    }
     if (total.majors.elapsed != 0 && total.majors.num != 0
-        && total.majors.elapsed > total.majors.num)
+        && total.majors.elapsed > total.majors.num) {
       currentMajorStdDev = stddev(total.majors.elapsed, total.majors.num, total.majors.sumDev);
+    }
 
     ActionStatsUpdator.update(total.minors, historical.minors);
     ActionStatsUpdator.update(total.majors, historical.majors);
@@ -228,9 +236,6 @@ public class TabletServerResource {
     return new TabletServerSummary(details, allTime, currentRes, currentOps);
   }
 
-  private static final int concurrentScans = Monitor.getContext().getConfiguration()
-      .getScanExecutors().stream().mapToInt(sec -> sec.maxThreads).sum();
-
   /**
    * Generates the server stats
    *
@@ -240,6 +245,9 @@ public class TabletServerResource {
   @GET
   public ServerStats getServerStats() {
 
+    final int concurrentScans = monitor.getContext().getConfiguration().getScanExecutors().stream()
+        .mapToInt(sec -> sec.maxThreads).sum();
+
     ServerStats stats = new ServerStats();
 
     stats.addStats(
@@ -320,7 +328,7 @@ public class TabletServerResource {
       String obscuredExtent = Base64.getEncoder().encodeToString(digester.digest());
       String displayExtent = String.format("[%s]", obscuredExtent);
 
-      String tableName = Tables.getPrintableTableInfoFromId(Monitor.getContext(), tableId);
+      String tableName = Tables.getPrintableTableInfoFromId(monitor.getContext(), tableId);
 
       currentOperations.add(
           new CurrentOperations(tableName, tableId, displayExtent, info.numEntries, info.ingestRate,
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
index 1a26b4b..42a28ca 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
@@ -31,6 +31,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import javax.inject.Inject;
 import javax.validation.constraints.Max;
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotNull;
@@ -67,6 +68,9 @@ public class WebViews {
 
   private static final Logger log = LoggerFactory.getLogger(WebViews.class);
 
+  @Inject
+  private Monitor monitor;
+
   /**
    * Get HTML for external CSS and JS resources from configuration. See ACCUMULO-4739
    *
@@ -74,10 +78,11 @@ public class WebViews {
    *          map of the MVC model
    */
   private void addExternalResources(Map<String,Object> model) {
-    AccumuloConfiguration conf = Monitor.getContext().getConfiguration();
+    AccumuloConfiguration conf = monitor.getContext().getConfiguration();
     String resourcesProperty = conf.get(Property.MONITOR_RESOURCES_EXTERNAL);
-    if (isEmpty(resourcesProperty))
+    if (isEmpty(resourcesProperty)) {
       return;
+    }
     List<String> monitorResources = new ArrayList<>();
     ObjectMapper objectMapper = new ObjectMapper();
     try {
@@ -98,8 +103,8 @@ public class WebViews {
 
     Map<String,Object> model = new HashMap<>();
     model.put("version", Constants.VERSION);
-    model.put("instance_name", Monitor.cachedInstanceName.get());
-    model.put("instance_id", Monitor.getContext().getInstanceID());
+    model.put("instance_name", monitor.getContext().getInstanceName());
+    model.put("instance_id", monitor.getContext().getInstanceID());
     addExternalResources(model);
     return model;
   }
@@ -293,7 +298,7 @@ public class WebViews {
       @PathParam("tableID") @NotNull @Pattern(regexp = ALPHA_NUM_REGEX_TABLE_ID) String tableID)
       throws TableNotFoundException {
 
-    String tableName = Tables.getTableName(Monitor.getContext(), TableId.of(tableID));
+    String tableName = Tables.getTableName(monitor.getContext(), TableId.of(tableID));
 
     Map<String,Object> model = getModel();
     model.put("title", "Table Status");
diff --git a/server/monitor/src/test/java/org/apache/accumulo/monitor/it/WebViewsIT.java b/server/monitor/src/test/java/org/apache/accumulo/monitor/it/WebViewsIT.java
index cc51b94..bdf7e3b 100644
--- a/server/monitor/src/test/java/org/apache/accumulo/monitor/it/WebViewsIT.java
+++ b/server/monitor/src/test/java/org/apache/accumulo/monitor/it/WebViewsIT.java
@@ -26,6 +26,7 @@ import java.io.OutputStream;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
 import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
 
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Application;
@@ -38,6 +39,7 @@ import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.monitor.Monitor;
+import org.apache.accumulo.monitor.Monitor.MonitorFactory;
 import org.apache.accumulo.monitor.view.WebViews;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.test.categories.MonitorTests;
@@ -46,6 +48,8 @@ import org.glassfish.jersey.client.ClientConfig;
 import org.glassfish.jersey.server.ResourceConfig;
 import org.glassfish.jersey.test.JerseyTest;
 import org.glassfish.jersey.test.TestProperties;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -66,6 +70,7 @@ public class WebViewsIT extends JerseyTest {
     enable(TestProperties.LOG_TRAFFIC);
     enable(TestProperties.DUMP_ENTITY);
     ResourceConfig config = new ResourceConfig(WebViews.class);
+    config.register(new MonitorFactory(monitor.get()));
     config.register(WebViewsIT.HashMapWriter.class);
     return config;
   }
@@ -76,6 +81,28 @@ public class WebViewsIT extends JerseyTest {
     config.register(WebViewsIT.HashMapWriter.class);
   }
 
+  private static AtomicReference<Monitor> monitor = new AtomicReference<>(null);
+
+  @BeforeClass
+  public static void createMocks() {
+    ServerContext contextMock = EasyMock.createMock(ServerContext.class);
+    expect(contextMock.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
+    expect(contextMock.getInstanceID()).andReturn("foo").atLeastOnce();
+    expect(contextMock.getInstanceName()).andReturn("foo").anyTimes();
+
+    Monitor monitorMock = EasyMock.createMock(Monitor.class);
+    expect(monitorMock.getContext()).andReturn(contextMock).anyTimes();
+
+    EasyMock.replay(contextMock, monitorMock);
+    monitor.set(monitorMock);
+  }
+
+  @AfterClass
+  public static void finishMocks() {
+    Monitor m = monitor.get();
+    verify(m.getContext(), m);
+  }
+
   /**
    * Expect to fail the constraint validation on the REST endpoint. The constraint is the
    * pre-defined word character class Pattern so passing a table name with punctuation will cause a
@@ -100,17 +127,9 @@ public class WebViewsIT extends JerseyTest {
    */
   @Test
   public void testGetTablesConstraintPassing() throws Exception {
-    ServerContext contextMock = EasyMock.createMock(ServerContext.class);
-    expect(contextMock.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    expect(contextMock.getInstanceID()).andReturn("foo").atLeastOnce();
-
-    PowerMock.mockStatic(Monitor.class);
-    expect(Monitor.getContext()).andReturn(contextMock).anyTimes();
-
     PowerMock.mockStatic(Tables.class);
-    expect(Tables.getTableName(contextMock, TableId.of("foo"))).andReturn("bar");
+    expect(Tables.getTableName(monitor.get().getContext(), TableId.of("foo"))).andReturn("bar");
     PowerMock.replayAll();
-    org.easymock.EasyMock.replay(contextMock);
 
     // Using the mocks we can verify that the getModel method gets called via debugger
     // however it's difficult to continue to mock through the jersey MVC code for the properly built
@@ -120,7 +139,6 @@ public class WebViewsIT extends JerseyTest {
     assertEquals("should return status 200", 200, output.getStatus());
     String responseBody = output.readEntity(String.class);
     assertTrue(responseBody.contains("tableID=foo") && responseBody.contains("table=bar"));
-    verify(contextMock);
   }
 
   /**
diff --git a/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java b/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
index 4361962..8abdb16 100644
--- a/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
+++ b/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
@@ -26,8 +26,10 @@ import org.apache.accumulo.core.master.thrift.Compacting;
 import org.apache.accumulo.core.master.thrift.RecoveryStatus;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.tables.CompactionsTypes;
 import org.apache.accumulo.monitor.rest.trace.RecoveryStatusInformation;
+import org.easymock.EasyMock;
 import org.junit.Test;
 
 public class TabletServerInformationTest {
@@ -66,7 +68,9 @@ public class TabletServerInformationTest {
     ts.setTableMap(Collections.singletonMap("tableId0", tableInfo));
     ts.setVersion("testVersion");
 
-    TabletServerInformation tsi = new TabletServerInformation(ts);
+    Monitor monitor = EasyMock.createMock(Monitor.class);
+
+    TabletServerInformation tsi = new TabletServerInformation(monitor, ts);
 
     assertEquals("tServerTestName:1234", tsi.server);
     assertEquals("tServerTestName:1234", tsi.hostname);