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"));