You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2018/10/22 17:14:59 UTC

[accumulo] branch master updated: Removed Accumulo class name conflict (#716)

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

mwalch 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 0a04bec  Removed Accumulo class name conflict (#716)
0a04bec is described below

commit 0a04becb111efbc9eb1b3e2a76680eac4edf5204
Author: Mike Walch <mw...@apache.org>
AuthorDate: Mon Oct 22 13:03:44 2018 -0400

    Removed Accumulo class name conflict (#716)
    
    * Class with same name exist in core & server-base
    * Renamed Accumulo class in server-base to ServerUtil
    * Updated code to use Accumulo.newClient() instead of impl code
---
 .../core/client/impl/AccumuloClientImpl.java       |  4 +-
 .../minicluster/impl/MiniAccumuloClusterImpl.java  | 10 ++---
 .../apache/accumulo/server/ServerConstants.java    |  2 +-
 .../org/apache/accumulo/server/ServerContext.java  | 10 ++---
 .../org/apache/accumulo/server/ServerInfo.java     |  2 +-
 .../server/{Accumulo.java => ServerUtil.java}      | 52 ++++++++++------------
 .../apache/accumulo/server/init/Initialize.java    |  6 +--
 .../org/apache/accumulo/server/AccumuloTest.java   | 10 ++---
 .../java/org/apache/accumulo/master/Master.java    | 14 +++---
 .../apache/accumulo/master/state/SetGoalState.java |  4 +-
 .../org/apache/accumulo/tracer/TraceServer.java    |  9 ++--
 11 files changed, 57 insertions(+), 66 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
index 1ef0d2a..59849b0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
@@ -22,6 +22,7 @@ import java.util.Objects;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -226,8 +227,7 @@ public class AccumuloClientImpl implements AccumuloClient {
   @Override
   public AccumuloClient changeUser(String principal, AuthenticationToken token)
       throws AccumuloSecurityException, AccumuloException {
-    return new AccumuloClientBuilderImpl().usingClientInfo(info()).usingToken(principal, token)
-        .build();
+    return Accumulo.newClient().usingClientInfo(info()).usingToken(principal, token).build();
   }
 
   public static class AccumuloClientBuilderImpl
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index be8285f..8eda2bb 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -54,6 +54,7 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -82,7 +83,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.master.state.SetGoalState;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
@@ -542,7 +543,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
-      Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(fs);
+      Path instanceIdPath = ServerUtil.getAccumuloInstanceIdPath(fs);
 
       String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, cc, hadoopConf);
       IZooReaderWriter zrw = new ZooReaderWriterFactory().getZooReaderWriter(
@@ -783,8 +784,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   @Override
   public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
-    return new AccumuloClientImpl.AccumuloClientBuilderImpl().usingClientInfo(getClientInfo())
-        .usingToken(user, token).build();
+    return Accumulo.newClient().usingClientInfo(getClientInfo()).usingToken(user, token).build();
   }
 
   @SuppressWarnings("deprecation")
@@ -797,7 +797,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   @Override
   public ClientInfo getClientInfo() {
     if (clientInfo == null) {
-      clientInfo = new AccumuloClientImpl.AccumuloClientBuilderImpl()
+      clientInfo = Accumulo.newClient()
           .usingProperties(config.getClientPropsFile().getAbsolutePath()).info();
     }
     return clientInfo;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
index 71780f5..9a53939 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
@@ -111,7 +111,7 @@ public class ServerConstants {
       try {
         currentIid = ZooUtil.getInstanceIDFromHdfs(path, conf);
         Path vpath = new Path(baseDir, VERSION_DIR);
-        currentVersion = Accumulo.getAccumuloPersistentVersion(
+        currentVersion = ServerUtil.getAccumuloPersistentVersion(
             vpath.getFileSystem(CachedConfiguration.getInstance()), vpath);
       } catch (Exception e) {
         if (ignore)
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 83cbe97..9670a70 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Objects;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -96,11 +97,7 @@ public class ServerContext extends ClientContext {
     SecurityUtil.serverLogin(info.getSiteConfiguration());
     log.info("Version " + Constants.VERSION);
     log.info("Instance " + info.getInstanceID());
-    try {
-      Accumulo.init(this, applicationName);
-    } catch (IOException e) {
-      throw new IllegalStateException(e);
-    }
+    ServerUtil.init(this, applicationName);
     MetricsSystemHelper.configure(applicationClassName);
     DistributedTrace.enable(hostname, applicationName,
         getServerConfFactory().getSystemConfiguration());
@@ -248,8 +245,7 @@ public class ServerContext extends ClientContext {
 
   public AccumuloClient getClient(String principal, AuthenticationToken token)
       throws AccumuloSecurityException, AccumuloException {
-    return new AccumuloClientImpl.AccumuloClientBuilderImpl().usingClientInfo(info)
-        .usingToken(principal, token).build();
+    return Accumulo.newClient().usingClientInfo(info).usingToken(principal, token).build();
   }
 
   public synchronized TableManager getTableManager() {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
index 6a4b42a..e15cace 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
@@ -86,7 +86,7 @@ public class ServerInfo implements ClientInfo {
     } catch (IOException e) {
       throw new IllegalStateException(e);
     }
-    Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(volumeManager);
+    Path instanceIdPath = ServerUtil.getAccumuloInstanceIdPath(volumeManager);
     instanceID = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, config);
     zooKeeperRoot = ZooUtil.getRoot(instanceID);
     zooKeepers = config.get(Property.INSTANCE_ZK_HOST);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
similarity index 87%
rename from server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
rename to server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
index 9db6d06..1491d91 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
@@ -47,9 +47,9 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class Accumulo {
+public class ServerUtil {
 
-  private static final Logger log = LoggerFactory.getLogger(Accumulo.class);
+  private static final Logger log = LoggerFactory.getLogger(ServerUtil.class);
 
   public static synchronized void updateAccumuloVersion(VolumeManager fs, int oldVersion) {
     for (Volume volume : fs.getVolumes()) {
@@ -104,14 +104,14 @@ public class Accumulo {
     return ServerConstants.getInstanceIdLocation(v);
   }
 
-  public static void init(ServerContext context, String application) throws IOException {
+  public static void init(ServerContext context, String application) {
     final AccumuloConfiguration conf = context.getConfiguration();
 
     log.info("{} starting", application);
     log.info("Instance {}", context.getInstanceID());
-    int dataVersion = Accumulo.getAccumuloPersistentVersion(context.getVolumeManager());
+    int dataVersion = ServerUtil.getAccumuloPersistentVersion(context.getVolumeManager());
     log.info("Data Version {}", dataVersion);
-    Accumulo.waitForZookeeperAndHdfs(context);
+    ServerUtil.waitForZookeeperAndHdfs(context);
 
     if (!(canUpgradeFromDataVersion(dataVersion))) {
       throw new RuntimeException("This version of accumulo (" + Constants.VERSION
@@ -159,33 +159,27 @@ public class Accumulo {
     return ServerConstants.NEEDS_UPGRADE.get(accumuloPersistentVersion);
   }
 
-  /**
-   *
-   */
   public static void monitorSwappiness(AccumuloConfiguration config) {
-    SimpleTimer.getInstance(config).schedule(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          String procFile = "/proc/sys/vm/swappiness";
-          File swappiness = new File(procFile);
-          if (swappiness.exists() && swappiness.canRead()) {
-            try (InputStream is = new FileInputStream(procFile)) {
-              byte[] buffer = new byte[10];
-              int bytes = is.read(buffer);
-              String setting = new String(buffer, 0, bytes, UTF_8);
-              setting = setting.trim();
-              if (bytes > 0 && Integer.parseInt(setting) > 10) {
-                log.warn("System swappiness setting is greater than ten ({})"
-                    + " which can cause time-sensitive operations to be delayed."
-                    + " Accumulo is time sensitive because it needs to maintain"
-                    + " distributed lock agreement.", setting);
-              }
+    SimpleTimer.getInstance(config).schedule(() -> {
+      try {
+        String procFile = "/proc/sys/vm/swappiness";
+        File swappiness = new File(procFile);
+        if (swappiness.exists() && swappiness.canRead()) {
+          try (InputStream is = new FileInputStream(procFile)) {
+            byte[] buffer = new byte[10];
+            int bytes = is.read(buffer);
+            String setting = new String(buffer, 0, bytes, UTF_8);
+            setting = setting.trim();
+            if (bytes > 0 && Integer.parseInt(setting) > 10) {
+              log.warn("System swappiness setting is greater than ten ({})"
+                  + " which can cause time-sensitive operations to be delayed."
+                  + " Accumulo is time sensitive because it needs to maintain"
+                  + " distributed lock agreement.", setting);
             }
           }
-        } catch (Throwable t) {
-          log.error("", t);
         }
+      } catch (Throwable t) {
+        log.error("", t);
       }
     }, 1000, 10 * 60 * 1000);
   }
@@ -260,7 +254,7 @@ public class Accumulo {
    */
   public static void abortIfFateTransactions(ServerContext context) {
     try {
-      final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<>(new ZooStore<>(
+      final ReadOnlyTStore<ServerUtil> fate = new ReadOnlyStore<>(new ZooStore<>(
           context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()));
       if (!(fate.list().isEmpty())) {
         throw new AccumuloException("Aborting upgrade because there are"
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 5150cc3..ac74a25 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -81,7 +81,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.constraints.MetadataConstraints;
@@ -865,10 +865,10 @@ public class Initialize implements KeywordExecutable {
             aBasePath, Property.INSTANCE_VOLUMES_REPLACEMENTS, Property.INSTANCE_VOLUMES);
     }
 
-    if (ServerConstants.DATA_VERSION != Accumulo.getAccumuloPersistentVersion(
+    if (ServerConstants.DATA_VERSION != ServerUtil.getAccumuloPersistentVersion(
         versionPath.getFileSystem(CachedConfiguration.getInstance()), versionPath)) {
       throw new IOException("Accumulo " + Constants.VERSION + " cannot initialize data version "
-          + Accumulo.getAccumuloPersistentVersion(fs));
+          + ServerUtil.getAccumuloPersistentVersion(fs));
     }
 
     initDirs(fs, uuid, uinitializedDirs.toArray(new String[uinitializedDirs.size()]), true);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
index 4e59eea..abf10cd 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
@@ -61,7 +61,7 @@ public class AccumuloTest {
     expect(fs.listStatus(path)).andReturn(files);
     replay(fs);
 
-    assertEquals(42, Accumulo.getAccumuloPersistentVersion(fs, path));
+    assertEquals(42, ServerUtil.getAccumuloPersistentVersion(fs, path));
   }
 
   @Test
@@ -69,7 +69,7 @@ public class AccumuloTest {
     expect(fs.listStatus(path)).andReturn(null);
     replay(fs);
 
-    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+    assertEquals(-1, ServerUtil.getAccumuloPersistentVersion(fs, path));
   }
 
   @Test
@@ -77,7 +77,7 @@ public class AccumuloTest {
     expect(fs.listStatus(path)).andReturn(new FileStatus[0]);
     replay(fs);
 
-    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+    assertEquals(-1, ServerUtil.getAccumuloPersistentVersion(fs, path));
   }
 
   @Test(expected = RuntimeException.class)
@@ -85,7 +85,7 @@ public class AccumuloTest {
     expect(fs.listStatus(path)).andThrow(new FileNotFoundException());
     replay(fs);
 
-    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+    assertEquals(-1, ServerUtil.getAccumuloPersistentVersion(fs, path));
   }
 
   @Test
@@ -134,6 +134,6 @@ public class AccumuloTest {
     expect(vm.create(newVersion2)).andReturn(fsdos2);
     replay(vm);
 
-    Accumulo.updateAccumuloVersion(vm, 7);
+    ServerUtil.updateAccumuloVersion(vm, 7);
   }
 }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 4d0baca..2732abf 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -96,7 +96,7 @@ import org.apache.accumulo.master.replication.MasterReplicationCoordinator;
 import org.apache.accumulo.master.replication.ReplicationDriver;
 import org.apache.accumulo.master.replication.WorkDriver;
 import org.apache.accumulo.master.state.TableCounts;
-import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
@@ -330,8 +330,8 @@ public class Master
     // introduce unnecessary complexity to try to make the master do it), but be aware
     // that the master is not the only thing that may alter zookeeper before starting.
 
-    final int accumuloPersistentVersion = Accumulo.getAccumuloPersistentVersion(fs);
-    if (Accumulo.persistentVersionNeedsUpgrade(accumuloPersistentVersion)) {
+    final int accumuloPersistentVersion = ServerUtil.getAccumuloPersistentVersion(fs);
+    if (ServerUtil.persistentVersionNeedsUpgrade(accumuloPersistentVersion)) {
       // This Master hasn't started Fate yet, so any outstanding transactions must be from before
       // the upgrade.
       // Change to Guava's Verify once we use Guava 17.
@@ -340,7 +340,7 @@ public class Master
             + " initialized prior to the Master transitioning to active. Please"
             + " save all logs and file a bug.");
       }
-      Accumulo.abortIfFateTransactions(getContext());
+      ServerUtil.abortIfFateTransactions(getContext());
       try {
         log.info("Upgrading zookeeper");
 
@@ -493,8 +493,8 @@ public class Master
     // we make sure we're only doing the rest of this method once so that we can signal to other
     // threads that an upgrade wasn't needed.
     if (upgradeMetadataRunning.compareAndSet(false, true)) {
-      final int accumuloPersistentVersion = Accumulo.getAccumuloPersistentVersion(fs);
-      if (Accumulo.persistentVersionNeedsUpgrade(accumuloPersistentVersion)) {
+      final int accumuloPersistentVersion = ServerUtil.getAccumuloPersistentVersion(fs);
+      if (ServerUtil.persistentVersionNeedsUpgrade(accumuloPersistentVersion)) {
         // sanity check that we passed the Fate verification prior to ZooKeeper upgrade, and that
         // Fate still hasn't been started.
         // Change both to use Guava's Verify once we use Guava 17.
@@ -533,7 +533,7 @@ public class Master
                 version++;
               }
               log.info("Updating persistent data version.");
-              Accumulo.updateAccumuloVersion(fs, accumuloPersistentVersion);
+              ServerUtil.updateAccumuloVersion(fs, accumuloPersistentVersion);
               log.info("Upgrade complete");
               waitForMetadataUpgrade.countDown();
             } catch (Exception ex) {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java b/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
index 8ea1284..a6cd28f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
@@ -22,7 +22,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SecurityUtil;
 
@@ -40,7 +40,7 @@ public class SetGoalState {
 
     ServerContext context = new ServerContext(new SiteConfiguration());
     SecurityUtil.serverLogin(context.getConfiguration());
-    Accumulo.waitForZookeeperAndHdfs(context);
+    ServerUtil.waitForZookeeperAndHdfs(context);
     context.getZooReaderWriter().putPersistentData(
         context.getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(UTF_8),
         NodeExistsPolicy.OVERWRITE);
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
index 8644a92..6e3b5e1 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
@@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -51,7 +52,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.AgeOffFilter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -273,8 +274,8 @@ public class TraceServer implements Watcher {
           at = token;
         }
 
-        accumuloClient = new AccumuloClientImpl.AccumuloClientBuilderImpl()
-            .usingClientInfo(context.getClientInfo()).usingToken(principal, at).build();
+        accumuloClient = Accumulo.newClient().usingClientInfo(context.getClientInfo())
+            .usingToken(principal, at).build();
         if (!accumuloClient.tableOperations().exists(tableName)) {
           accumuloClient.tableOperations().create(tableName);
           IteratorSetting setting = new IteratorSetting(10, "ageoff", AgeOffFilter.class.getName());
@@ -404,7 +405,7 @@ public class TraceServer implements Watcher {
     ServerContext context = new ServerContext(opts.getSiteConfiguration());
     loginTracer(context.getConfiguration());
     MetricsSystemHelper.configure(TraceServer.class.getSimpleName());
-    Accumulo.init(context, app);
+    ServerUtil.init(context, app);
     TraceServer server = new TraceServer(context, opts.getAddress());
     try {
       server.run();