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 <[email protected]>
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"));