You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/08/13 20:02:55 UTC

[GitHub] mikewalch closed pull request #599: Fixes #582 - Construct ServerContext once and pass it

mikewalch closed pull request #599: Fixes #582 - Construct ServerContext once and pass it
URL: https://github.com/apache/accumulo/pull/599
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 a2428d964c..0d1dca3691 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
@@ -55,8 +55,6 @@
 
   private static final Logger log = LoggerFactory.getLogger(ServerContext.class);
 
-  private static ServerContext serverContextInstance = null;
-
   private final ServerInfo info;
   private TableManager tableManager;
   private UniqueNameAllocator nameAllocator;
@@ -66,7 +64,7 @@
   private String hostname = null;
   private AuthenticationTokenSecretManager secretManager;
 
-  private ServerContext() {
+  public ServerContext() {
     this(new ServerInfo());
   }
 
@@ -83,28 +81,6 @@ public ServerContext(ClientInfo info) {
     this(new ServerInfo(info));
   }
 
-  synchronized public static ServerContext getInstance() {
-    if (serverContextInstance == null) {
-      serverContextInstance = new ServerContext();
-    }
-    return serverContextInstance;
-  }
-
-  synchronized public static ServerContext getInstance(ClientInfo info) {
-    if (serverContextInstance == null) {
-      serverContextInstance = new ServerContext(info);
-    }
-    return serverContextInstance;
-  }
-
-  synchronized public static ServerContext getInstance(String instanceName, String zooKeepers,
-      int zooKeepersSessionTimeOut) {
-    if (serverContextInstance == null) {
-      serverContextInstance = new ServerContext(instanceName, zooKeepers, zooKeepersSessionTimeOut);
-    }
-    return serverContextInstance;
-  }
-
   public void setupServer(String appName, String appClassName, String hostname) {
     applicationName = appName;
     applicationClassName = appClassName;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
index 8d3c24582a..1192814794 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
@@ -25,9 +25,9 @@
 
   public ServerContext getServerContext() {
     if (instance == null) {
-      return ServerContext.getInstance();
+      return new ServerContext();
     }
-    return ServerContext.getInstance(getClientInfo());
+    return new ServerContext(getClientInfo());
   }
 
   public ClientOnDefaultTable(String table) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
index d231310baa..10363207b2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
@@ -25,8 +25,8 @@
 
   public ServerContext getServerContext() {
     if (instance == null) {
-      return ServerContext.getInstance();
+      return new ServerContext();
     }
-    return ServerContext.getInstance(getClientInfo());
+    return new ServerContext(getClientInfo());
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
index b204226666..dab53e2425 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
@@ -30,8 +30,8 @@ public ClientContext getClientContext() {
 
   public ServerContext getServerContext() {
     if (instance == null) {
-      return ServerContext.getInstance();
+      return new ServerContext();
     }
-    return ServerContext.getInstance(getClientInfo());
+    return new ServerContext(getClientInfo());
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
index a0069bdbf7..be67117f52 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
@@ -25,7 +25,7 @@
 public class ConfigSanityCheck implements KeywordExecutable {
 
   public static void main(String[] args) {
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     context.getServerConfFactory().getSystemConfiguration();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
index 86a3c7ccdb..108cf082f9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
@@ -97,6 +97,10 @@ public ServerConfigurationFactory(ServerContext context) {
     addInstanceToCaches(instanceID);
   }
 
+  public ServerContext getServerContext() {
+    return context;
+  }
+
   void setZooCacheFactory(ZooCacheFactory zcf) {
     this.zcf = zcf;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index ce9eb3eb86..4ac08bc69c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -108,7 +108,7 @@ private static boolean isValidColumn(ColumnUpdate cu) {
 
   @Override
   public List<Short> check(Environment env, Mutation mutation) {
-    final ServerContext context = ((SystemEnvironment)env).getServerContext();
+    final ServerContext context = ((SystemEnvironment) env).getServerContext();
 
     ArrayList<Short> violations = null;
 
@@ -231,8 +231,8 @@ private static boolean isValidColumn(ColumnUpdate cu) {
             long tid = Long.parseLong(tidString);
 
             try {
-              if (otherTidCount > 0 || !dataFiles.equals(loadedFiles)
-                  || !getArbitrator(context).transactionAlive(Constants.BULK_ARBITRATOR_TYPE, tid)) {
+              if (otherTidCount > 0 || !dataFiles.equals(loadedFiles) || !getArbitrator(context)
+                  .transactionAlive(Constants.BULK_ARBITRATOR_TYPE, tid)) {
                 violations = addViolation(violations, 8);
               }
             } catch (Exception ex) {
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 059ac580f1..3c0557dc45 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
@@ -391,7 +391,7 @@ private boolean initialize(Opts opts, String instanceNamePath, VolumeManager fs,
       return false;
     }
 
-    final ServerContext context = ServerContext.getInstance();
+    final ServerContext context = new ServerContext();
 
     // When we're using Kerberos authentication, we need valid credentials to perform
     // initialization. If the user provided some, use them.
@@ -932,7 +932,7 @@ public void execute(final String[] args) {
 
       if (opts.resetSecurity) {
         log.info("Resetting security on accumulo.");
-        ServerContext context = ServerContext.getInstance();
+        ServerContext context = new ServerContext();
         if (isInitialized(fs)) {
           if (!opts.forceResetSecurity) {
             ConsoleReader c = getConsoleReader();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilter.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilter.java
index 244e9f9dfa..c5980aedd9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilter.java
@@ -84,10 +84,10 @@ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> op
     }
 
     bulkTxStatusCache = new HashMap<>();
-    arbitrator = getArbitrator();
+    arbitrator = getArbitrator(((SystemIteratorEnvironment) env).getServerContext());
   }
 
-  protected Arbitrator getArbitrator() {
-    return new ZooArbitrator(ServerContext.getInstance());
+  protected Arbitrator getArbitrator(ServerContext context) {
+    return new ZooArbitrator(context);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java
new file mode 100644
index 0000000000..9cd7f38a8e
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.iterators;
+
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.server.ServerContext;
+
+public interface SystemIteratorEnvironment extends IteratorEnvironment {
+
+  ServerContext getServerContext();
+
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index e1bdc814fb..46a6e59902 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -69,7 +69,7 @@
    */
   @Deprecated
   public void init(ServerConfigurationFactory conf) {
-    init(ServerContext.getInstance());
+    init(conf.getServerContext());
   }
 
   /**
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index ad95fdac0c..8217a670f3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -300,7 +300,7 @@ public static synchronized ProblemReports getInstance(ServerContext context) {
   }
 
   public static void main(String args[]) throws Exception {
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     getInstance(context).printProblems();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
index d6a31fc4cc..d241d5793c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
@@ -56,7 +56,7 @@ public static void main(String[] args) {
       zk.getZooKeeper().addAuthInfo("digest", ("accumulo:" + opts.auth).getBytes(UTF_8));
     }
 
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
 
     try {
       for (String child : zk.getChildren(root)) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
index e5c15c21ed..4bc8f85e66 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
@@ -43,7 +43,7 @@ public String description() {
 
   @Override
   public void execute(final String[] args) throws KeeperException, InterruptedException {
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     System.out.println("monitor: " + MonitorUtil.getLocation(context));
     System.out.println("masters: " + context.getMasterLocations());
     System.out.println("zookeepers: " + context.getZooKeepers());
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index 8af8f35533..219ddd46ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -37,7 +37,7 @@
 public class ListVolumesUsed {
 
   public static void main(String[] args) throws Exception {
-    listVolumes(ServerContext.getInstance());
+    listVolumes(new ServerContext());
   }
 
   private static String getTableURI(String rootTabletDir) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
index bc1ca89c2f..ed415e3cab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
@@ -46,7 +46,7 @@ public String description() {
 
   @Override
   public void execute(String[] args) throws Exception {
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     AccumuloConfiguration config = context.getServerConfFactory().getSystemConfiguration();
     Authenticator authenticator = AccumuloVFSClassLoader.getClassLoader()
         .loadClass(config.get(Property.INSTANCE_SECURITY_AUTHENTICATOR))
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
index dedaff1bb0..3228abf5b6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
@@ -41,7 +41,7 @@
 
   public static void main(String[] args) throws Exception {
 
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
     Opts opts = new Opts();
     opts.parseArgs(TabletServerLocks.class.getName(), args);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
index 1b1a10743c..6ac817f9cf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
@@ -68,7 +68,7 @@ public void execute(final String[] args) throws Exception {
     String baseDir = ServerConstants.getBaseUris()[0];
     System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id"))
         + " to lookup accumulo instance");
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     if (opts.servers == null) {
       opts.servers = context.getZooKeepers();
     }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
index c101f16368..473a320fe9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
@@ -21,7 +21,6 @@
 import java.util.HashMap;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.client.impl.BaseIteratorEnvironment;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
@@ -32,7 +31,9 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -53,7 +54,7 @@ public boolean transactionComplete(String type, long tid) throws Exception {
 
   static class TestMetadataBulkLoadFilter extends MetadataBulkLoadFilter {
     @Override
-    protected Arbitrator getArbitrator() {
+    protected Arbitrator getArbitrator(ServerContext context) {
       return new TestArbitrator();
     }
   }
@@ -99,20 +100,14 @@ public void testBasic() throws IOException {
     put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/file9", "8");
     put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/fileA", "2");
 
-    TestMetadataBulkLoadFilter iter = new TestMetadataBulkLoadFilter();
-    iter.init(new SortedMapIterator(tm1), new HashMap<>(), new BaseIteratorEnvironment() {
-
-      @Override
-      public boolean isFullMajorCompaction() {
-        return false;
-      }
-
-      @Override
-      public IteratorScope getIteratorScope() {
-        return IteratorScope.majc;
-      }
-    });
+    SystemIteratorEnvironment env = EasyMock.createMock(SystemIteratorEnvironment.class);
+    EasyMock.expect(env.getServerContext()).andReturn(null);
+    EasyMock.expect(env.isFullMajorCompaction()).andReturn(false);
+    EasyMock.expect(env.getIteratorScope()).andReturn(IteratorScope.majc);
+    EasyMock.replay(env);
 
+    TestMetadataBulkLoadFilter iter = new TestMetadataBulkLoadFilter();
+    iter.init(new SortedMapIterator(tm1), new HashMap<>(), env);
     iter.seek(new Range(), new ArrayList<>(), false);
 
     TreeMap<Key,Value> actual = new TreeMap<>();
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index aa777e2eb8..c897de420b 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -141,7 +141,7 @@ public static void main(String[] args) {
     final String app = "gc";
     Opts opts = new Opts();
     opts.parseArgs(app, args);
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     context.setupServer(app, SimpleGarbageCollector.class.getName(), opts.getAddress());
     try {
       SimpleGarbageCollector gc = new SimpleGarbageCollector(opts, context);
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 0ddf8292bd..6a2a6f8bcf 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
@@ -1552,7 +1552,7 @@ public static void main(String[] args) throws Exception {
     final String app = "master";
     ServerOpts opts = new ServerOpts();
     opts.parseArgs(app, args);
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     context.setupServer(app, Master.class.getName(), opts.getAddress());
     try {
       Master master = new Master(context);
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 017598f619..57a235acd4 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
@@ -40,7 +40,7 @@ public static void main(String[] args) throws Exception {
     }
     SecurityUtil.serverLogin(SiteConfiguration.getInstance());
 
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     Accumulo.waitForZookeeperAndHdfs(context.getVolumeManager());
     ZooReaderWriter.getInstance().putPersistentData(
         context.getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(UTF_8),
diff --git a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
index 1e51257d13..bc21f8b592 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
@@ -77,7 +77,7 @@ public static void main(String[] args) throws Exception {
 
     AdminUtil<Master> admin = new AdminUtil<>();
 
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     final String zkRoot = context.getZooKeeperRoot();
     String path = zkRoot + Constants.ZFATE;
     String masterPath = zkRoot + Constants.ZMASTER_LOCK;
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 a048c1d216..624a2d31b9 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
@@ -434,7 +434,7 @@ public static void main(String[] args) throws Exception {
     final String app = "monitor";
     ServerOpts opts = new ServerOpts();
     opts.parseArgs(app, args);
-    Monitor.context = ServerContext.getInstance();
+    Monitor.context = new ServerContext();
     context.setupServer(app, Monitor.class.getName(), opts.getAddress());
     try {
       config = context.getServerConfFactory();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
index c886b0f56d..88432e545b 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
@@ -144,7 +144,7 @@ public int hashCode() {
     public MonitorLocation get() {
       // lazily set up path and zooCache (see comment in constructor)
       if (this.context == null) {
-        this.context = ServerContext.getInstance();
+        this.context = new ServerContext();
         this.path = context.getZooKeeperRoot() + Constants.ZMONITOR_LOG4J_ADDR;
         this.zooCache = context.getZooCache();
       }
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 4c10406bd7..43a5e67247 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
@@ -403,7 +403,7 @@ public static void main(String[] args) throws Exception {
     ServerOpts opts = new ServerOpts();
     opts.parseArgs(app, args);
     loginTracer(SiteConfiguration.getInstance());
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     MetricsSystemHelper.configure(TraceServer.class.getSimpleName());
     Accumulo.init(context.getVolumeManager(), context.getInstanceID(),
         context.getServerConfFactory(), app);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
index 7774cfa5ba..ce0365b122 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
@@ -41,6 +41,7 @@
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration.ParsedIteratorConfig;
 import org.apache.accumulo.tserver.data.ServerConditionalMutation;
@@ -67,7 +68,8 @@
 
   private Map<ByteSequence,MergedIterConfig> mergedIterCache = new HashMap<>();
 
-  ConditionCheckerContext(CompressedIterators compressedIters, TableConfiguration tableConf) {
+  ConditionCheckerContext(ServerContext serverContext, CompressedIterators compressedIters,
+      TableConfiguration tableConf) {
     this.compressedIters = compressedIters;
 
     ParsedIteratorConfig pic = tableConf.getParsedIteratorConfig(IteratorScope.scan);
@@ -78,7 +80,7 @@
 
     classCache = new HashMap<>();
 
-    tie = new TabletIteratorEnvironment(IteratorScope.scan, tableConf);
+    tie = new TabletIteratorEnvironment(serverContext, IteratorScope.scan, tableConf);
   }
 
   SortedKeyValueIterator<Key,Value> buildIterator(SortedKeyValueIterator<Key,Value> systemIter,
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
index c25fde800c..4a4e093000 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
@@ -33,13 +33,16 @@
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
 import org.apache.accumulo.tserver.FileManager.ScanFileManager;
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
 import org.apache.hadoop.fs.Path;
 
-public class TabletIteratorEnvironment implements IteratorEnvironment {
+public class TabletIteratorEnvironment implements SystemIteratorEnvironment {
 
+  private final ServerContext context;
   private final ScanFileManager trm;
   private final IteratorScope scope;
   private final boolean fullMajorCompaction;
@@ -52,10 +55,12 @@
   private SamplerConfiguration samplerConfig;
   private boolean enableSampleForDeepCopy;
 
-  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config) {
+  public TabletIteratorEnvironment(ServerContext context, IteratorScope scope,
+      AccumuloConfiguration config) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
 
+    this.context = context;
     this.scope = scope;
     this.trm = null;
     this.config = config;
@@ -65,13 +70,14 @@ public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration conf
     this.topLevelIterators = new ArrayList<>();
   }
 
-  private TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config,
-      ScanFileManager trm, Map<FileRef,DataFileValue> files, Authorizations authorizations,
-      SamplerConfigurationImpl samplerConfig,
+  private TabletIteratorEnvironment(ServerContext context, IteratorScope scope,
+      AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files,
+      Authorizations authorizations, SamplerConfigurationImpl samplerConfig,
       ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
 
+    this.context = context;
     this.scope = scope;
     this.trm = trm;
     this.config = config;
@@ -88,18 +94,19 @@ private TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration con
     this.topLevelIterators = topLevelIterators;
   }
 
-  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config,
-      ScanFileManager trm, Map<FileRef,DataFileValue> files, Authorizations authorizations,
-      SamplerConfigurationImpl samplerConfig) {
-    this(scope, config, trm, files, authorizations, samplerConfig, new ArrayList<>());
+  public TabletIteratorEnvironment(ServerContext context, IteratorScope scope,
+      AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files,
+      Authorizations authorizations, SamplerConfigurationImpl samplerConfig) {
+    this(context, scope, config, trm, files, authorizations, samplerConfig, new ArrayList<>());
   }
 
-  public TabletIteratorEnvironment(IteratorScope scope, boolean fullMajC,
+  public TabletIteratorEnvironment(ServerContext context, IteratorScope scope, boolean fullMajC,
       AccumuloConfiguration config, MajorCompactionReason reason) {
     if (scope != IteratorScope.majc)
       throw new IllegalArgumentException(
           "Tried to set maj compaction type when scope was " + scope);
 
+    this.context = context;
     this.scope = scope;
     this.trm = null;
     this.config = config;
@@ -192,7 +199,12 @@ public IteratorEnvironment cloneWithSamplingEnabled() {
       throw new SampleNotPresentException();
     }
 
-    return new TabletIteratorEnvironment(scope, config, trm, files, authorizations, sci,
+    return new TabletIteratorEnvironment(context, scope, config, trm, files, authorizations, sci,
         topLevelIterators);
   }
+
+  @Override
+  public ServerContext getServerContext() {
+    return context;
+  }
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 0451a8629d..831d9a3f5b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -1313,7 +1313,7 @@ private void checkConditions(Map<KeyExtent,List<ServerConditionalMutation>> upda
           .iterator();
 
       final CompressedIterators compressedIters = new CompressedIterators(symbols);
-      ConditionCheckerContext checkerContext = new ConditionCheckerContext(compressedIters,
+      ConditionCheckerContext checkerContext = new ConditionCheckerContext(context, compressedIters,
           confFactory.getTableConfiguration(cs.tableId));
 
       while (iter.hasNext()) {
@@ -3338,7 +3338,7 @@ public static void main(String[] args) throws Exception {
     final String app = "tserver";
     ServerOpts opts = new ServerOpts();
     opts.parseArgs(app, args);
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     context.setupServer(app, TabletServer.class.getSimpleName(), opts.getAddress());
     try {
       final TabletServer server = new TabletServer(context);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index 904af5fc94..fdfbca8e2c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -351,10 +351,10 @@ private void compactLocalityGroup(String lgName, Set<ByteSequence> columnFamilie
 
       TabletIteratorEnvironment iterEnv;
       if (env.getIteratorScope() == IteratorScope.majc)
-        iterEnv = new TabletIteratorEnvironment(IteratorScope.majc, !propogateDeletes, acuTableConf,
-            getMajorCompactionReason());
+        iterEnv = new TabletIteratorEnvironment(context, IteratorScope.majc, !propogateDeletes,
+            acuTableConf, getMajorCompactionReason());
       else if (env.getIteratorScope() == IteratorScope.minc)
-        iterEnv = new TabletIteratorEnvironment(IteratorScope.minc, acuTableConf);
+        iterEnv = new TabletIteratorEnvironment(context, IteratorScope.minc, acuTableConf);
       else
         throw new IllegalArgumentException();
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index 0ddd362546..e75918f676 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -191,9 +191,9 @@ public boolean isCurrent() {
 
     MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent());
 
-    TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(IteratorScope.scan,
-        tablet.getTableConfiguration(), fileManager, files, options.getAuthorizations(),
-        samplerConfig);
+    TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(
+        tablet.getTabletServer().getContext(), IteratorScope.scan, tablet.getTableConfiguration(),
+        fileManager, files, options.getAuthorizations(), samplerConfig);
 
     statsIterator = new StatsIterator(multiIter, TabletServer.seekCount,
         tablet.getScannedCounter());
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ListBulkCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ListBulkCommand.java
index e62b078815..2c38aa6bd5 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ListBulkCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ListBulkCommand.java
@@ -52,7 +52,7 @@ public int execute(final String fullCommand, final CommandLine cl, final Shell s
 
     MasterMonitorInfo stats;
     MasterClientService.Iface client = null;
-    ServerContext context = ServerContext.getInstance(shellState.getContext().getClientInfo());
+    ServerContext context = new ServerContext(shellState.getContext().getClientInfo());
     while (true) {
       try {
         client = MasterClient.getConnectionWithRetry(context);
diff --git a/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java b/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
index da9f3374eb..65ec9809c6 100644
--- a/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
@@ -40,7 +40,7 @@
   public static void main(String[] args) throws Exception {
     MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     while (true) {
       try {
         client = MasterClient.getConnectionWithRetry(context);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index 1898e983e6..74684fb40c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -285,7 +285,7 @@ private void verifySame(SortedMap<FileRef,DataFileValue> datafileSizes,
   }
 
   public static void main(String[] args) throws Exception {
-    new SplitRecoveryIT().run(ServerContext.getInstance());
+    new SplitRecoveryIT().run(new ServerContext());
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 63eea7fe77..d8547da73b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -99,7 +99,7 @@ synchronized public void halt(TInfo tinfo, TCredentials credentials, String lock
   public static void main(String[] args) throws Exception {
     Random random = new Random(System.currentTimeMillis() % 1000);
     int port = random.nextInt(30000) + 2000;
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     TransactionWatcher watcher = new TransactionWatcher(context);
     final ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
     Processor<Iface> processor = new Processor<>(tch);
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index 7bd4c37437..549f335db2 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -309,7 +309,7 @@ public static void main(String[] args) throws Exception {
     // modify metadata
     int zkTimeOut = (int) DefaultConfiguration.getInstance()
         .getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-    ServerContext context = ServerContext.getInstance(opts.iname, opts.keepers, zkTimeOut);
+    ServerContext context = new ServerContext(opts.iname, opts.keepers, zkTimeOut);
     TransactionWatcher watcher = new TransactionWatcher(context);
     ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
     Processor<Iface> processor = new Processor<>(tch);
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index 283d7e6267..39b33e64f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -58,7 +58,7 @@ public void testSystemCredentials() throws Exception {
 
   public static void main(final String[] args)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    ServerContext context = ServerContext.getInstance();
+    ServerContext context = new ServerContext();
     Credentials creds = null;
     String badInstanceID = SystemCredentials.class.getName();
     if (args.length < 2)


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services