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

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


The following commit(s) were added to refs/heads/main by this push:
     new d1ea90fa85 Remove SingletonManager and related code (#5282)
d1ea90fa85 is described below

commit d1ea90fa8503fc31a6bedb1d267e38debb18106d
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Wed Mar 19 17:26:02 2025 -0400

    Remove SingletonManager and related code (#5282)
    
    Remove last remaining static singleton by attaching tablet locator code
    to the ClientContext
---
 .../accumulo/core/clientImpl/ClientContext.java    |  35 ++--
 .../core/clientImpl/ClientTabletCache.java         |  80 +-------
 .../accumulo/core/singletons/SingletonManager.java | 190 -------------------
 .../core/singletons/SingletonReservation.java      |  71 -------
 .../accumulo/core/singletons/SingletonService.java |  33 ----
 .../core/singletons/SingletonManagerTest.java      | 203 ---------------------
 .../org/apache/accumulo/server/ServerContext.java  |   3 +-
 .../org/apache/accumulo/server/ServerInfo.java     |   3 -
 .../apache/accumulo/server/init/Initialize.java    |   3 -
 .../org/apache/accumulo/server/util/Admin.java     |   4 -
 .../org/apache/accumulo/server/util/ECAdmin.java   |   4 -
 .../org/apache/accumulo/server/util/ZooZap.java    |   4 -
 .../accumulo/manager/state/SetGoalState.java       |  18 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |   3 +-
 .../apache/accumulo/test/BatchWriterIterator.java  |   3 +-
 .../accumulo/test/functional/AccumuloClientIT.java |  12 --
 .../accumulo/test/functional/ManagerApiIT.java     |   5 -
 17 files changed, 30 insertions(+), 644 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 3dfe1aed95..491670ade1 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -95,8 +95,6 @@ import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanServerInfo;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
@@ -137,6 +135,7 @@ public class ClientContext implements AccumuloClient {
   private ConditionalWriterConfig conditionalWriterConfig;
   private final AccumuloConfiguration accumuloConf;
   private final Configuration hadoopConf;
+  private final HashMap<TableId,ClientTabletCache> tabletCaches = new 
HashMap<>();
 
   // These fields are very frequently accessed (each time a connection is 
created) and expensive to
   // compute, so cache them.
@@ -156,7 +155,6 @@ public class ClientContext implements AccumuloClient {
   private final TableOperationsImpl tableops;
   private final NamespaceOperations namespaceops;
   private InstanceOperations instanceops = null;
-  private final SingletonReservation singletonReservation;
   private final ThreadPools clientThreadPools;
   private ThreadPoolExecutor cleanupThreadPool;
   private ThreadPoolExecutor scannerReadaheadPool;
@@ -223,13 +221,11 @@ public class ClientContext implements AccumuloClient {
   }
 
   /**
-   * Create a client context with the provided configuration. Legacy client 
code must provide a
-   * no-op SingletonReservation to preserve behavior prior to 2.x. Clients 
since 2.x should call
-   * Accumulo.newClient() builder, which will create a client reservation in
-   * {@link ClientBuilderImpl#buildClient}
+   * Create a client context with the provided configuration. Clients should 
call
+   * Accumulo.newClient() builder
    */
-  public ClientContext(SingletonReservation reservation, ClientInfo info,
-      AccumuloConfiguration serverConf, UncaughtExceptionHandler ueh) {
+  public ClientContext(ClientInfo info, AccumuloConfiguration serverConf,
+      UncaughtExceptionHandler ueh) {
     this.info = info;
     this.hadoopConf = info.getHadoopConf();
 
@@ -254,7 +250,6 @@ public class ClientContext implements AccumuloClient {
         () -> SaslConnectionParams.from(getConfiguration(), 
getCredentials().getToken()), 100,
         MILLISECONDS);
     scanServerSelectorSupplier = memoize(this::createScanServerSelector);
-    this.singletonReservation = Objects.requireNonNull(reservation);
     this.tableops = new TableOperationsImpl(this);
     this.namespaceops = new NamespaceOperationsImpl(this, tableops);
     this.serverPaths = Suppliers.memoize(() -> new 
ServiceLockPaths(this.getZooCache()));
@@ -807,7 +802,6 @@ public class ClientContext implements AccumuloClient {
       if (cleanupThreadPool != null) {
         cleanupThreadPool.shutdown(); // wait for shutdown tasks to execute
       }
-      singletonReservation.close();
     }
   }
 
@@ -840,16 +834,10 @@ public class ClientContext implements AccumuloClient {
     }
 
     public static AccumuloClient buildClient(ClientBuilderImpl<AccumuloClient> 
cbi) {
-      SingletonReservation reservation = 
SingletonManager.getClientReservation();
-      try {
-        // ClientContext closes reservation unless a RuntimeException is thrown
-        ClientInfo info = cbi.getClientInfo();
-        var config = 
ClientConfConverter.toAccumuloConf(info.getClientProperties());
-        return new ClientContext(reservation, info, config, 
cbi.getUncaughtExceptionHandler());
-      } catch (RuntimeException e) {
-        reservation.close();
-        throw e;
-      }
+      // ClientContext closes reservation unless a RuntimeException is thrown
+      ClientInfo info = cbi.getClientInfo();
+      var config = 
ClientConfConverter.toAccumuloConf(info.getClientProperties());
+      return new ClientContext(info, config, 
cbi.getUncaughtExceptionHandler());
     }
 
     public static Properties buildProps(ClientBuilderImpl<Properties> cbi) {
@@ -1100,6 +1088,11 @@ public class ClientContext implements AccumuloClient {
     return namespaces;
   }
 
+  public HashMap<TableId,ClientTabletCache> tabletCaches() {
+    ensureOpen();
+    return tabletCaches;
+  }
+
   private static Set<String> createPersistentWatcherPaths() {
     Set<String> pathsToWatch = new HashSet<>();
     for (String path : Set.of(Constants.ZCOMPACTORS, Constants.ZDEADTSERVERS, 
Constants.ZGC_LOCK,
diff --git 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
index 61b96c5cdc..adcd5fc618 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientTabletCache.java
@@ -34,22 +34,17 @@ import 
org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.InvalidTabletHostingRequestException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TabletAvailability;
-import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.MetadataCachedTabletObtainer;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonService;
 import org.apache.accumulo.core.util.Interner;
 import org.apache.accumulo.core.util.Timer;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 
-import com.google.common.base.Preconditions;
-
 /**
  * Client side cache of information about Tablets. Currently, a tablet prev 
end row is cached and
  * locations are cached if they exist.
@@ -189,66 +184,21 @@ public abstract class ClientTabletCache {
    */
   public abstract void invalidateCache(ClientContext context, String server);
 
-  private static class InstanceKey {
-    final InstanceId instanceId;
-    final TableId tableId;
-
-    InstanceKey(InstanceId instanceId, TableId table) {
-      this.instanceId = instanceId;
-      this.tableId = table;
-    }
-
-    @Override
-    public int hashCode() {
-      return instanceId.hashCode() + tableId.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof InstanceKey) {
-        return equals((InstanceKey) o);
-      }
-      return false;
-    }
-
-    public boolean equals(InstanceKey lk) {
-      return instanceId.equals(lk.instanceId) && tableId.equals(lk.tableId);
-    }
-
-  }
-
-  private static final HashMap<InstanceKey,ClientTabletCache> instances = new 
HashMap<>();
-  private static boolean enabled = true;
-
-  public static synchronized void clearInstances() {
+  public static synchronized void clearInstances(ClientContext context) {
+    final var instances = context.tabletCaches();
     for (ClientTabletCache locator : instances.values()) {
       locator.isValid = false;
     }
     instances.clear();
   }
 
-  static synchronized boolean isEnabled() {
-    return enabled;
-  }
-
-  static synchronized void disable() {
-    clearInstances();
-    enabled = false;
-  }
-
-  static synchronized void enable() {
-    enabled = true;
-  }
-
   public long getTabletHostingRequestCount() {
     return 0L;
   }
 
   public static synchronized ClientTabletCache getInstance(ClientContext 
context, TableId tableId) {
-    Preconditions.checkState(enabled, "The Accumulo singleton that that tracks 
tablet locations is "
-        + "disabled. This is likely caused by all AccumuloClients being closed 
or garbage collected");
-    InstanceKey key = new InstanceKey(context.getInstanceID(), tableId);
-    ClientTabletCache tl = instances.get(key);
+    final var caches = context.tabletCaches();
+    ClientTabletCache tl = caches.get(tableId);
     if (tl == null) {
       MetadataCachedTabletObtainer mlo = new MetadataCachedTabletObtainer();
 
@@ -263,32 +213,12 @@ public abstract class ClientTabletCache {
             getInstance(context, AccumuloTable.METADATA.tableId()), mlo,
             context.getTServerLockChecker());
       }
-      instances.put(key, tl);
+      caches.put(tableId, tl);
     }
 
     return tl;
   }
 
-  static {
-    SingletonManager.register(new SingletonService() {
-
-      @Override
-      public boolean isEnabled() {
-        return ClientTabletCache.isEnabled();
-      }
-
-      @Override
-      public void enable() {
-        ClientTabletCache.enable();
-      }
-
-      @Override
-      public void disable() {
-        ClientTabletCache.disable();
-      }
-    });
-  }
-
   public static class CachedTablets {
 
     private final List<CachedTablet> cachedTablets;
diff --git 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java 
b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
deleted file mode 100644
index 9a380598a9..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonManager.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * 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
- *
- *   https://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.core.singletons;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-/**
- * This class automates management of static singletons that maintain state 
for Accumulo clients.
- * Historically, Accumulo client code that used Connector had no control over 
these singletons. The
- * new AccumuloClient API that replaces Connector is closeable. When all 
AccumuloClients are closed
- * then resources used by the singletons are released. This class coordinates 
releasing those
- * resources.
- *
- * <p>
- * This class is intermediate solution to resource management. Ideally there 
would be no static
- * state and AccumuloClients would own all of their state and clean it up on 
close. If
- * AccumuloClient is not closable at inception, then it is harder to make it 
closable later. If
- * AccumuloClient is not closable, then its hard to remove the static state. 
This class enables
- * making AccumuloClient closable at inception so that static state can be 
removed later.
- *
- */
-public class SingletonManager {
-
-  private static final Logger log = 
LoggerFactory.getLogger(SingletonManager.class);
-
-  /**
-   * These enums determine the behavior of the SingletonManager.
-   *
-   */
-  public enum Mode {
-    /**
-     * In this mode singletons are disabled when the number of active client 
reservations goes to
-     * zero.
-     */
-    CLIENT,
-    /**
-     * In this mode singletons are never disabled, unless the CLOSED mode is 
entered.
-     */
-    SERVER,
-    /**
-     * In this mode singletons are permanently disabled and entering this mode 
prevents
-     * transitioning to other modes.
-     */
-    CLOSED
-
-  }
-
-  private static long reservations;
-  private static Mode mode;
-  private static boolean enabled;
-  private static List<SingletonService> services;
-
-  @VisibleForTesting
-  static void reset() {
-    reservations = 0;
-    mode = Mode.CLIENT;
-    enabled = true;
-    services = new ArrayList<>();
-  }
-
-  static {
-    reset();
-  }
-
-  private static void enable(SingletonService service) {
-    try {
-      service.enable();
-    } catch (RuntimeException e) {
-      log.error("Failed to enable singleton service", e);
-    }
-  }
-
-  private static void disable(SingletonService service) {
-    try {
-      service.disable();
-    } catch (RuntimeException e) {
-      log.error("Failed to disable singleton service", e);
-    }
-  }
-
-  /**
-   * Register a static singleton that should be disabled and enabled as needed.
-   */
-  public static synchronized void register(SingletonService service) {
-    if (enabled && !service.isEnabled()) {
-      enable(service);
-    }
-
-    if (!enabled && service.isEnabled()) {
-      disable(service);
-    }
-
-    services.add(service);
-  }
-
-  /**
-   * This method should be called when creating Accumulo clients using the 
public API. Accumulo
-   * clients created internally within Accumulo code should probably call
-   * {@link SingletonReservation#noop()} instead. While a client holds a 
reservation, singleton
-   * services are enabled.
-   *
-   * @return A reservation that must be closed when the AccumuloClient is 
closed.
-   */
-  public static synchronized SingletonReservation getClientReservation() {
-    Preconditions.checkState(reservations >= 0);
-    reservations++;
-    transition();
-    return new SingletonReservation();
-  }
-
-  static synchronized void releaseReservation() {
-    Preconditions.checkState(reservations > 0);
-    reservations--;
-    transition();
-  }
-
-  @VisibleForTesting
-  public static long getReservationCount() {
-    return reservations;
-  }
-
-  /**
-   * Change how singletons are managed. The default mode is {@link Mode#CLIENT}
-   */
-  public static synchronized void setMode(Mode mode) {
-    if (SingletonManager.mode == mode) {
-      return;
-    }
-    if (SingletonManager.mode == Mode.CLOSED) {
-      throw new IllegalStateException("Cannot leave closed mode once entered");
-    }
-
-    /*
-     * Always allow transition to closed and only allow transition to 
client/connector when the
-     * current mode is not server.
-     */
-    if (SingletonManager.mode != Mode.SERVER || mode == Mode.CLOSED) {
-      SingletonManager.mode = mode;
-    }
-    transition();
-  }
-
-  @VisibleForTesting
-  public static synchronized Mode getMode() {
-    return mode;
-  }
-
-  private static void transition() {
-    if (enabled) {
-      // if we're in an enabled state AND
-      // the mode is CLOSED or there are no active clients,
-      // then disable everything
-      if (mode == Mode.CLOSED || (mode == Mode.CLIENT && reservations == 0)) {
-        services.forEach(SingletonManager::disable);
-        enabled = false;
-      }
-    } else {
-      // if we're in a disabled state AND
-      // the mode is SERVER or if there are active clients,
-      // then enable everything
-      if (mode == Mode.SERVER || (mode == Mode.CLIENT && reservations > 0)) {
-        services.forEach(SingletonManager::enable);
-        enabled = true;
-      }
-    }
-  }
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
 
b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
deleted file mode 100644
index e7448fd5f7..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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
- *
- *   https://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.core.singletons;
-
-import java.lang.ref.Cleaner.Cleanable;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.util.cleaner.CleanerUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @see SingletonManager#getClientReservation()
- */
-public class SingletonReservation implements AutoCloseable {
-
-  private static final Logger log = 
LoggerFactory.getLogger(SingletonReservation.class);
-
-  // AtomicBoolean so cleaner doesn't need to synchronize to reliably read
-  private final AtomicBoolean closed = new AtomicBoolean(false);
-  private final Cleanable cleanable;
-
-  public SingletonReservation() {
-    cleanable = CleanerUtil.unclosed(this, AccumuloClient.class, closed, log, 
null);
-  }
-
-  @Override
-  public void close() {
-    if (closed.compareAndSet(false, true)) {
-      // deregister cleanable, but it won't run because it checks
-      // the value of closed first, which is now true
-      cleanable.clean();
-      SingletonManager.releaseReservation();
-    }
-  }
-
-  private static class NoopSingletonReservation extends SingletonReservation {
-    NoopSingletonReservation() {
-      super.closed.set(true);
-      // deregister the cleaner
-      super.cleanable.clean();
-    }
-
-  }
-
-  private static final SingletonReservation NOOP = new 
NoopSingletonReservation();
-
-  /**
-   * @return A reservation where the close method is a no-op.
-   */
-  public static SingletonReservation noop() {
-    return NOOP;
-  }
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonService.java 
b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonService.java
deleted file mode 100644
index 5438c78bd6..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonService.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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
- *
- *   https://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.core.singletons;
-
-/**
- * The {@link SingletonManager} uses this interface to enable and disable 
singleton services.
- *
- * @see SingletonManager#register(SingletonService)
- */
-public interface SingletonService {
-
-  public boolean isEnabled();
-
-  public void enable();
-
-  public void disable();
-}
diff --git 
a/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
deleted file mode 100644
index 9c7d1dd8a3..0000000000
--- 
a/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * 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
- *
- *   https://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.core.singletons;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-public class SingletonManagerTest {
-
-  TestService service1;
-  TestService service2;
-
-  @BeforeEach
-  public void setup() {
-    SingletonManager.reset();
-    assertEquals(0, SingletonManager.getReservationCount());
-
-    service1 = new TestService(true);
-    service2 = new TestService(false);
-
-    SingletonManager.register(service1);
-    SingletonManager.register(service2);
-
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    assertEquals(Mode.CLIENT, SingletonManager.getMode());
-  }
-
-  @Test
-  public void testClient() {
-    SingletonReservation resv1 = SingletonManager.getClientReservation();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-
-    SingletonReservation resv2 = SingletonManager.getClientReservation();
-
-    assertEquals(2, SingletonManager.getReservationCount());
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    resv1.close();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    // calling close again should have no effect
-    resv1.close();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    resv2.close();
-
-    assertEquals(0, SingletonManager.getReservationCount());
-    assertEquals(new TestService(false, 0, 1), service1);
-    assertEquals(new TestService(false, 1, 1), service2);
-
-    SingletonReservation resv3 = SingletonManager.getClientReservation();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-    assertEquals(new TestService(true, 1, 1), service1);
-    assertEquals(new TestService(true, 2, 1), service2);
-
-    resv3.close();
-
-    assertEquals(0, SingletonManager.getReservationCount());
-    assertEquals(new TestService(false, 1, 2), service1);
-    assertEquals(new TestService(false, 2, 2), service2);
-  }
-
-  @Test
-  public void testServerPreventsDisable() {
-
-    SingletonManager.setMode(Mode.SERVER);
-    assertEquals(Mode.SERVER, SingletonManager.getMode());
-
-    SingletonReservation resv1 = SingletonManager.getClientReservation();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-
-    SingletonReservation resv2 = SingletonManager.getClientReservation();
-
-    assertEquals(2, SingletonManager.getReservationCount());
-
-    resv1.close();
-    resv2.close();
-
-    assertEquals(0, SingletonManager.getReservationCount());
-
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    // can not leave server mode, so this should have no effect
-    SingletonManager.setMode(Mode.CLIENT);
-    assertEquals(Mode.SERVER, SingletonManager.getMode());
-
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-  }
-
-  @Test
-  public void testServerEnables() {
-    SingletonReservation resv1 = SingletonManager.getClientReservation();
-    resv1.close();
-
-    assertEquals(new TestService(false, 0, 1), service1);
-    assertEquals(new TestService(false, 1, 1), service2);
-
-    // this should enable services
-    SingletonManager.setMode(Mode.SERVER);
-    assertEquals(Mode.SERVER, SingletonManager.getMode());
-
-    assertEquals(new TestService(true, 1, 1), service1);
-    assertEquals(new TestService(true, 2, 1), service2);
-
-    // can not leave server mode, so this should have no effect
-    SingletonManager.setMode(Mode.CLIENT);
-    assertEquals(Mode.SERVER, SingletonManager.getMode());
-
-    assertEquals(new TestService(true, 1, 1), service1);
-    assertEquals(new TestService(true, 2, 1), service2);
-  }
-
-  private static class TestService implements SingletonService {
-
-    boolean enabled;
-    int enables = 0;
-    int disables = 0;
-
-    TestService(boolean enabled) {
-      this.enabled = enabled;
-    }
-
-    TestService(boolean enabled, int enables, int disables) {
-      this.enabled = enabled;
-      this.enables = enables;
-      this.disables = disables;
-    }
-
-    @Override
-    public boolean isEnabled() {
-      return enabled;
-    }
-
-    @Override
-    public void enable() {
-      assertFalse(enabled);
-      enabled = true;
-      enables++;
-
-    }
-
-    @Override
-    public void disable() {
-      assertTrue(enabled);
-      enabled = false;
-      disables++;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof TestService) {
-        TestService ots = (TestService) o;
-        return enabled == ots.enabled && enables == ots.enables && disables == 
ots.disables;
-      }
-      return false;
-    }
-
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public String toString() {
-      return "enabled:" + enabled + " enables:" + enables + " disables:" + 
disables;
-    }
-  }
-}
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 fa8150a8c7..5077705505 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
@@ -56,7 +56,6 @@ import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
-import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.spi.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -111,7 +110,7 @@ public class ServerContext extends ClientContext {
   }
 
   private ServerContext(ServerInfo info) {
-    super(SingletonReservation.noop(), info, info.getSiteConfiguration(), 
Threads.UEH);
+    super(info, info.getSiteConfiguration(), Threads.UEH);
     this.info = info;
     serverDirs = info.getServerDirs();
 
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 fd0dcbde94..617e26b8de 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
@@ -38,8 +38,6 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.zookeeper.ZooSession;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
@@ -122,7 +120,6 @@ public class ServerInfo implements ClientInfo {
   private ServerInfo(SiteConfiguration siteConfig, Function<ServerInfo,String> 
zkHostsFunction,
       ToIntFunction<ServerInfo> zkTimeoutFunction, Function<ServerInfo,String> 
instanceNameFunction,
       Function<ServerInfo,InstanceId> instanceIdFunction) {
-    SingletonManager.setMode(Mode.SERVER);
     this.siteConfig = requireNonNull(siteConfig);
     requireNonNull(zkHostsFunction);
     requireNonNull(zkTimeoutFunction);
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 b3df97de5a..8d7a8b4bfc 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
@@ -43,8 +43,6 @@ import 
org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooSession;
@@ -553,7 +551,6 @@ public class Initialize implements KeywordExecutable {
       log.error("ZooKeeper error when trying to get Volume configuration", e);
       success = false;
     } finally {
-      SingletonManager.setMode(Mode.CLOSED);
       if (!success) {
         System.exit(-1);
       }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java 
b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index 9d9cf95f81..7da6b43de9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -88,8 +88,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.Halt;
@@ -575,8 +573,6 @@ public class Admin implements KeywordExecutable {
     } catch (Exception e) {
       log.error("{}", e.getMessage(), e);
       System.exit(3);
-    } finally {
-      SingletonManager.setMode(Mode.CLOSED);
     }
   }
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java 
b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
index ea67ba7977..710568ff29 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
@@ -31,8 +31,6 @@ import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.compaction.RunningCompaction;
@@ -132,8 +130,6 @@ public class ECAdmin implements KeywordExecutable {
     } catch (Exception e) {
       log.error("{}", e.getMessage(), e);
       System.exit(1);
-    } finally {
-      SingletonManager.setMode(Mode.CLOSED);
     }
   }
 
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java 
b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
index 2922718010..16e13a84b3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
@@ -30,8 +30,6 @@ import 
org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector;
 import org.apache.accumulo.core.lock.ServiceLockPaths.ResourceGroupPredicate;
 import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.start.spi.KeywordExecutable;
@@ -92,8 +90,6 @@ public class ZooZap implements KeywordExecutable {
         SecurityUtil.serverLogin(siteConf);
       }
       zap(context, args);
-    } finally {
-      SingletonManager.setMode(Mode.CLOSED);
     }
   }
 
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
index ee2bac5bb5..b0fa8f83b9 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
@@ -24,8 +24,6 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SecurityUtil;
 
@@ -41,16 +39,12 @@ public class SetGoalState {
       System.exit(-1);
     }
 
-    try {
-      var siteConfig = SiteConfiguration.auto();
-      SecurityUtil.serverLogin(siteConfig);
-      try (var context = new ServerContext(siteConfig)) {
-        context.waitForZookeeperAndHdfs();
-        
context.getZooSession().asReaderWriter().putPersistentData(Constants.ZMANAGER_GOAL_STATE,
-            args[0].getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
-      }
-    } finally {
-      SingletonManager.setMode(Mode.CLOSED);
+    var siteConfig = SiteConfiguration.auto();
+    SecurityUtil.serverLogin(siteConfig);
+    try (var context = new ServerContext(siteConfig)) {
+      context.waitForZookeeperAndHdfs();
+      
context.getZooSession().asReaderWriter().putPersistentData(Constants.ZMANAGER_GOAL_STATE,
+          args[0].getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     }
   }
 
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 495e10723f..5a71c6cad2 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
@@ -330,7 +330,8 @@ public class TabletServer extends AbstractServer implements 
TabletHostingServer
     this.resourceManager = new TabletServerResourceManager(context, this);
 
     
watchCriticalScheduledTask(context.getScheduledExecutor().scheduleWithFixedDelay(
-        ClientTabletCache::clearInstances, jitter(), jitter(), 
TimeUnit.MILLISECONDS));
+        () -> ClientTabletCache.clearInstances(context), jitter(), jitter(),
+        TimeUnit.MILLISECONDS));
     walMarker = new WalStateManager(context);
 
     if (aconf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
diff --git 
a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java 
b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
index fecb79fb37..8948a2ee66 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.clientImpl.ClientTabletCache;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -217,7 +218,7 @@ public class BatchWriterIterator extends WrappingIterator {
         if (firstWrite) {
           batchWriter.flush();
           if (clearCacheAfterFirstWrite) {
-            ClientTabletCache.clearInstances();
+            ClientTabletCache.clearInstances((ClientContext) accumuloClient);
           }
           if (splitAfterFirstWrite) {
             SortedSet<Text> splits = new TreeSet<>();
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
index ea533fdb48..9556f65025 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
@@ -40,8 +40,6 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Test;
@@ -145,12 +143,7 @@ public class AccumuloClientIT extends 
AccumuloClusterHarness {
 
     Scanner scanner;
 
-    assertEquals(1, SingletonManager.getReservationCount());
-    assertEquals(Mode.CLIENT, SingletonManager.getMode());
-
     try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {
-      assertEquals(2, SingletonManager.getReservationCount());
-
       c.tableOperations().create(tableName);
 
       try (BatchWriter writer = c.createBatchWriter(tableName)) {
@@ -165,10 +158,7 @@ public class AccumuloClientIT extends 
AccumuloClusterHarness {
     // scanner created from closed client should fail
     expectClosed(() -> scanner.iterator().next());
 
-    assertEquals(1, SingletonManager.getReservationCount());
-
     AccumuloClient c = Accumulo.newClient().from(getClientProps()).build();
-    assertEquals(2, SingletonManager.getReservationCount());
 
     // ensure client created after everything was closed works
     Scanner scanner2 = c.createScanner(tableName, Authorizations.EMPTY);
@@ -183,8 +173,6 @@ public class AccumuloClientIT extends 
AccumuloClusterHarness {
 
     c.close();
 
-    assertEquals(1, SingletonManager.getReservationCount());
-
     expectClosed(() -> c.createScanner(tableName, Authorizations.EMPTY));
     expectClosed(() -> c.createConditionalWriter(tableName));
     expectClosed(() -> c.createBatchWriter(tableName));
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
index 8001bc77ca..593f382dfd 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
@@ -43,8 +43,6 @@ import 
org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
@@ -71,9 +69,6 @@ public class ManagerApiIT extends SharedMiniClusterBase {
 
   @BeforeAll
   public static void setup() throws Exception {
-    // need to pretend to be a server, so we can bypass all of
-    // the singleton resource management in this test
-    SingletonManager.setMode(Mode.SERVER);
     SharedMiniClusterBase.startMiniCluster();
     rootUser = new Credentials(getPrincipal(), getToken());
     regularUser = new Credentials("regularUser", new 
PasswordToken("regularUser"));


Reply via email to