From be22deed47dc7db3635823e30ba7fe498b79a5aa Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Fri, 10 Jan 2025 15:54:56 -0500 Subject: [PATCH] Clean up ZooKeeper tooling (#5192) * Replace existing ZooSession with a new one that is a facade for ZooKeeper. The methods from ZooKeeper we need are placed on it and internally it maintains a delegate ZooKeeper instance that handles automatically creating a new ZooKeeper session (client object) when the old session has died. This enables us to maintain fewer ZooKeeper objects by keeping only one at a time per ClientContext/ServerContext, and to reduce complexity substantially, where it was hard to reason about which ZooKeeper session we were using at any given moment. This no longer requires passing around ZooKeeper objects via ZooReader that called to the old ZooSession to construct a ZooKeeper session on demand. The new ZooSession is now a singleton field whose lifecycle is part of the context, and ZooReader/ZooReaderWriter are substantially simplified. * Lazily construct objects in ServerInfo/ClientInfoImpl to simplify the implementation for the various use cases, and to ensure that we don't create more objects than needed. * Improve debugging information to tie the ZooSession instances with their purpose (for example, for use with ServerContext, or for use with ClientContext, with the user's name) * Get rid of ZooCacheFactory in favor of a lazily constructed ZooCache instance in the ClientContext, to make it more clear when a ZooCache is being shared or reused, and to remove a static singleton * Move instanceId and instanceName lookup logic into ZooUtil * Make ZookeeperLockChecker use its own ZooSession, because it is still a static singleton and must continue to operate after the context that constructed it is closed (should be fixed when #2301 is done) * Perform some minor improvements to ZooCache to simplify its constructors now that it uses ZooSession, and to change the type of external watchers to Consumer, so they aren't as easily confused with actual ZooKeeper watchers * Improve a lot of ZooKeeper related test code Potential future work after this could include: * Roll ZooReader and ZooReaderWriter functions directly into ZooSession * Add support to ZooSession for more ZooKeeper APIs * Handle KeeperException thrown from delegate that signals the session is disconnected, instead of relying only on the verifyConnected() method in ZooSession to update the delegate * Handle InterruptedExceptions directly in ZooSession, so they don't propagate everywhere in the code in ways that are inconvenient to handle This fixes #5124, #2299, and #2298 --- .../core/clientImpl/ClientContext.java | 88 +++-- .../accumulo/core/clientImpl/ClientInfo.java | 22 +- .../core/clientImpl/ClientInfoImpl.java | 61 ++-- .../clientImpl/InstanceOperationsImpl.java | 16 - .../core/clientImpl/ZookeeperLockChecker.java | 9 +- .../apache/accumulo/core/fate/AdminUtil.java | 44 +-- .../apache/accumulo/core/fate/ZooStore.java | 43 +-- .../core/fate/zookeeper/ZooCache.java | 58 ++-- .../core/fate/zookeeper/ZooCacheFactory.java | 128 ------- .../core/fate/zookeeper/ZooReader.java | 38 +-- .../core/fate/zookeeper/ZooReaderWriter.java | 39 +-- .../core/fate/zookeeper/ZooSession.java | 252 -------------- .../accumulo/core/fate/zookeeper/ZooUtil.java | 103 ++++-- .../accumulo/core/lock/ServiceLock.java | 12 +- .../core/metadata/schema/TabletsMetadata.java | 2 +- .../accumulo/core/util/MonitorUtil.java | 2 +- .../compaction/ExternalCompactionUtil.java | 2 +- .../accumulo/core/zookeeper/ZooSession.java | 312 ++++++++++++++++++ .../clientImpl/ZookeeperLockCheckerTest.java | 37 +-- .../fate/zookeeper/ZooCacheFactoryTest.java | 83 ----- .../core/fate/zookeeper/ZooCacheTest.java | 27 +- .../fate/zookeeper/ZooReaderWriterTest.java | 39 +-- .../core/fate/zookeeper/ZooSessionTest.java | 43 --- .../core/fate/zookeeper/ZooUtilTest.java | 39 +++ .../core/zookeeper/ZooSessionTest.java | 62 ++++ minicluster/pom.xml | 4 - .../standalone/StandaloneAccumuloCluster.java | 22 +- .../MiniAccumuloClusterImpl.java | 96 ++---- .../apache/accumulo/server/ServerContext.java | 42 +-- .../apache/accumulo/server/ServerInfo.java | 209 +++++++----- .../accumulo/server/cli/ServerUtilOpts.java | 10 +- .../server/conf/store/impl/ZooPropLoader.java | 5 +- .../server/conf/store/impl/ZooPropStore.java | 25 +- .../server/conf/util/ZooInfoViewer.java | 88 ++--- .../server/conf/util/ZooPropEditor.java | 9 +- .../server/conf/util/ZooPropUtils.java | 84 ----- .../accumulo/server/init/Initialize.java | 6 +- .../server/init/ZooKeeperInitializer.java | 2 +- .../accumulo/server/log/WalStateManager.java | 2 +- .../server/manager/LiveTServerSet.java | 23 +- .../server/manager/state/DeadServerList.java | 4 +- .../metadata/RootTabletMutatorImpl.java | 2 +- .../server/metadata/ServerAmpleImpl.java | 4 +- .../server/security/SecurityOperation.java | 6 +- .../ZooAuthenticationKeyDistributor.java | 29 +- .../ZooAuthenticationKeyWatcher.java | 5 +- .../handler/KerberosAuthenticator.java | 6 +- .../security/handler/ZKAuthenticator.java | 13 +- .../server/security/handler/ZKAuthorizor.java | 11 +- .../security/handler/ZKPermHandler.java | 4 +- .../accumulo/server/tables/TableManager.java | 14 +- .../server/tablets/UniqueNameAllocator.java | 9 +- .../apache/accumulo/server/util/Admin.java | 5 +- .../accumulo/server/util/ChangeSecret.java | 113 ++++--- .../server/util/DeleteZooInstance.java | 42 +-- .../accumulo/server/util/DumpZookeeper.java | 30 +- .../accumulo/server/util/ListInstances.java | 56 ++-- .../server/util/RestoreZookeeper.java | 28 +- .../server/util/ServiceStatusCmd.java | 2 +- .../server/util/TabletServerLocks.java | 2 +- .../apache/accumulo/server/util/ZooZap.java | 35 +- .../checkCommand/RootMetadataCheckRunner.java | 4 +- .../checkCommand/TableLocksCheckRunner.java | 6 +- .../zookeeper/DistributedWorkQueue.java | 2 +- .../server/zookeeper/TransactionWatcher.java | 10 +- .../accumulo/server/MockServerContext.java | 16 +- ...ccumuloConfigurationIsPropertySetTest.java | 2 +- .../conf/NamespaceConfigurationTest.java | 5 +- .../server/conf/SystemConfigurationTest.java | 14 +- .../conf/store/impl/PropStoreEventTest.java | 32 +- .../conf/store/impl/ZooPropLoaderTest.java | 34 +- .../conf/store/impl/ZooPropStoreTest.java | 45 +-- .../server/conf/util/ZooInfoViewerTest.java | 77 +++-- .../server/conf/util/ZooPropUtilsTest.java | 68 ---- .../accumulo/server/init/InitializeTest.java | 38 ++- .../accumulo/server/rpc/TServerUtilsTest.java | 8 +- .../ZooAuthenticationKeyDistributorTest.java | 88 ++--- .../ZooAuthenticationKeyWatcherTest.java | 73 ++-- .../security/handler/ZKAuthenticatorTest.java | 16 +- .../server/util/ServiceStatusCmdTest.java | 120 +++---- .../coordinator/CompactionCoordinator.java | 6 +- .../apache/accumulo/compactor/Compactor.java | 15 +- .../java/org/apache/accumulo/gc/GCRun.java | 2 +- .../accumulo/gc/SimpleGarbageCollector.java | 2 +- .../org/apache/accumulo/manager/Manager.java | 24 +- .../manager/ManagerClientServiceHandler.java | 2 +- .../apache/accumulo/manager/ManagerTime.java | 2 +- .../metrics/fate/FateMetricValues.java | 2 +- .../manager/metrics/fate/FateMetrics.java | 2 +- .../manager/recovery/RecoveryManager.java | 2 +- .../accumulo/manager/state/MergeStats.java | 2 +- .../accumulo/manager/state/SetGoalState.java | 2 +- .../accumulo/manager/tableOps/Utils.java | 19 +- .../tableOps/compact/CompactRange.java | 4 +- .../tableOps/compact/CompactionDriver.java | 2 +- .../compact/cancel/CancelCompactions.java | 2 +- .../tableOps/delete/PreDeleteTable.java | 2 +- .../PopulateZookeeperWithNamespace.java | 2 +- .../namespace/rename/RenameNamespace.java | 2 +- .../manager/tableOps/rename/RenameTable.java | 2 +- .../manager/tserverOps/ShutdownTServer.java | 2 +- .../manager/upgrade/PreUpgradeValidation.java | 8 +- .../manager/upgrade/UpgradeCoordinator.java | 4 +- .../manager/upgrade/Upgrader10to11.java | 2 +- .../manager/upgrade/Upgrader11to12.java | 11 +- .../compact/CompactionDriverTest.java | 102 +++--- .../manager/upgrade/Upgrader10to11Test.java | 71 ++-- .../manager/upgrade/Upgrader11to12Test.java | 22 +- .../org/apache/accumulo/monitor/Monitor.java | 8 +- .../apache/accumulo/tserver/ScanServer.java | 6 +- .../apache/accumulo/tserver/TabletServer.java | 11 +- .../accumulo/tserver/tablet/Tablet.java | 5 +- .../apache/accumulo/test/ExistingMacIT.java | 2 +- .../apache/accumulo/test/FateSummaryIT.java | 10 +- .../ScanServerConcurrentTabletScanIT.java | 2 +- .../test/ScanServerGroupConfigurationIT.java | 10 +- .../apache/accumulo/test/ScanServerIT.java | 2 +- .../test/ScanServerMetadataEntriesIT.java | 2 +- .../test/ScanServerMultipleScansIT.java | 2 +- .../accumulo/test/ScanServerShutdownIT.java | 2 +- ...hriftServerBindsBeforeZooKeeperLockIT.java | 4 +- .../accumulo/test/conf/PropStoreConfigIT.java | 2 +- .../conf/store/PropCacheCaffeineImplZkIT.java | 45 +-- .../test/conf/store/PropStoreZooKeeperIT.java | 55 ++- .../test/conf/store/ZooBasedConfigIT.java | 37 +-- .../accumulo/test/fate/zookeeper/FateIT.java | 69 ++-- .../test/fate/zookeeper/ZooMutatorIT.java | 8 +- .../test/functional/BackupManagerIT.java | 4 +- .../test/functional/CacheTestClean.java | 18 +- .../test/functional/CacheTestReader.java | 63 ++-- .../test/functional/CacheTestWriter.java | 178 +++++----- .../test/functional/FateConcurrencyIT.java | 16 +- .../test/functional/FunctionalTestUtils.java | 3 +- .../test/functional/GarbageCollectorIT.java | 4 +- .../functional/HalfDeadServerWatcherIT.java | 4 +- .../test/functional/ManagerApiIT.java | 171 +++------- .../test/functional/SplitRecoveryIT.java | 4 +- .../test/functional/WatchTheWatchCountIT.java | 4 +- .../test/functional/ZombieTServer.java | 4 +- .../accumulo/test/lock/ServiceLockIT.java | 88 ++--- .../test/performance/NullTserver.java | 2 +- .../zookeeper/ZooKeeperTestingServer.java | 34 +- 142 files changed, 2040 insertions(+), 2425 deletions(-) delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java delete mode 100644 core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java create mode 100644 core/src/main/java/org/apache/accumulo/core/zookeeper/ZooSession.java delete mode 100644 core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactoryTest.java delete mode 100644 core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooSessionTest.java create mode 100644 core/src/test/java/org/apache/accumulo/core/zookeeper/ZooSessionTest.java delete mode 100644 server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java delete mode 100644 server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java 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 67fec0bfbcc..702936aee92 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 @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Suppliers.memoize; import static com.google.common.base.Suppliers.memoizeWithExpiration; -import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; @@ -44,6 +43,7 @@ import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -79,8 +79,6 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat; -import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockData; @@ -106,12 +104,11 @@ import org.apache.accumulo.core.util.tables.TableZooHelper; import org.apache.accumulo.core.util.threads.ThreadPools; import org.apache.accumulo.core.util.threads.Threads; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Suppliers; - /** * This class represents any essential configuration and credentials needed to initiate RPC * operations throughout the code. It is intended to represent a shared object that contains these @@ -126,14 +123,12 @@ public class ClientContext implements AccumuloClient { private static final Logger log = LoggerFactory.getLogger(ClientContext.class); private final ClientInfo info; - private InstanceId instanceId; - private final ZooReader zooReader; - private final ZooCache zooCache; + private final Supplier zooCache; private Credentials creds; private BatchWriterConfig batchWriterConfig; private ConditionalWriterConfig conditionalWriterConfig; - private final AccumuloConfiguration serverConf; + private final AccumuloConfiguration accumuloConf; private final Configuration hadoopConf; // These fields are very frequently accessed (each time a connection is created) and expensive to @@ -158,6 +153,9 @@ public class ClientContext implements AccumuloClient { private ThreadPoolExecutor cleanupThreadPool; private ThreadPoolExecutor scannerReadaheadPool; + private final AtomicBoolean zooKeeperOpened = new AtomicBoolean(false); + private final Supplier zooSession; + private void ensureOpen() { if (closed) { throw new IllegalStateException("This client was closed."); @@ -224,13 +222,19 @@ public ClientContext(SingletonReservation reservation, ClientInfo info, AccumuloConfiguration serverConf, UncaughtExceptionHandler ueh) { this.info = info; this.hadoopConf = info.getHadoopConf(); - zooReader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut()); - zooCache = - new ZooCacheFactory().getZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut()); - this.serverConf = serverConf; + + this.zooSession = memoize(() -> { + var zk = info + .getZooKeeperSupplier(getClass().getSimpleName() + "(" + info.getPrincipal() + ")").get(); + zooKeeperOpened.set(true); + return zk; + }); + + this.zooCache = memoize(() -> new ZooCache(getZooSession())); + this.accumuloConf = serverConf; timeoutSupplier = memoizeWithExpiration( () -> getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT), 100, MILLISECONDS); - sslSupplier = Suppliers.memoize(() -> SslConnectionParams.forClient(getConfiguration())); + sslSupplier = memoize(() -> SslConnectionParams.forClient(getConfiguration())); saslSupplier = memoizeWithExpiration( () -> SaslConnectionParams.from(getConfiguration(), getCredentials().getToken()), 100, MILLISECONDS); @@ -328,7 +332,7 @@ public synchronized void setCredentials(Credentials newCredentials) { */ public AccumuloConfiguration getConfiguration() { ensureOpen(); - return serverConf; + return accumuloConf; } /** @@ -520,7 +524,7 @@ public List getManagerLocations() { timer = Timer.startNew(); } - Optional sld = zooCache.getLockData(zLockManagerPath); + Optional sld = getZooCache().getLockData(zLockManagerPath); String location = null; if (sld.isPresent()) { location = sld.orElseThrow().getAddressString(ThriftService.MANAGER); @@ -546,26 +550,7 @@ public List getManagerLocations() { */ public InstanceId getInstanceID() { ensureOpen(); - if (instanceId == null) { - // lookup by name - final String instanceName = info.getInstanceName(); - String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName; - byte[] data = zooCache.get(instanceNamePath); - if (data == null) { - throw new RuntimeException( - "Instance name " + instanceName + " does not exist in zookeeper. " - + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list."); - } - String instanceIdString = new String(data, UTF_8); - // verify that the instanceId found via the instanceName actually exists as an instance - if (zooCache.get(Constants.ZROOT + "/" + instanceIdString) == null) { - throw new RuntimeException("Instance id " + instanceIdString - + (instanceName == null ? "" : " pointed to by the name " + instanceName) - + " does not exist in zookeeper"); - } - instanceId = InstanceId.of(instanceIdString); - } - return instanceId; + return info.getInstanceId(); } public String getZooKeeperRoot() { @@ -605,7 +590,7 @@ public int getZooKeepersSessionTimeOut() { public ZooCache getZooCache() { ensureOpen(); - return zooCache; + return zooCache.get(); } private TableZooHelper tableZooHelper; @@ -860,6 +845,9 @@ public AuthenticationToken token() { @Override public synchronized void close() { closed = true; + if (zooKeeperOpened.get()) { + zooSession.get().close(); + } if (thriftTransportPool != null) { thriftTransportPool.shutdown(); } @@ -880,7 +868,7 @@ public static class ClientBuilderImpl SslOptions, SaslOptions, ClientFactory, FromOptions { private Properties properties = new Properties(); - private AuthenticationToken token = null; + private Optional tokenOpt = Optional.empty(); private final Function,T> builderFunction; private UncaughtExceptionHandler ueh = null; @@ -889,12 +877,9 @@ public ClientBuilderImpl(Function,T> builderFunction) { } private ClientInfo getClientInfo() { - if (token != null) { - ClientProperty.validate(properties, false); - return new ClientInfoImpl(properties, token); - } - ClientProperty.validate(properties); - return new ClientInfoImpl(properties); + // validate the token in the properties if not provided here + ClientProperty.validate(properties, tokenOpt.isEmpty()); + return new ClientInfoImpl(properties, tokenOpt); } private UncaughtExceptionHandler getUncaughtExceptionHandler() { @@ -1065,7 +1050,7 @@ public ConnectionOptions as(CharSequence principal, AuthenticationToken token } setProperty(ClientProperty.AUTH_PRINCIPAL, principal.toString()); ClientProperty.setAuthenticationToken(properties, token); - this.token = token; + this.tokenOpt = Optional.of(token); return this; } @@ -1089,9 +1074,9 @@ public ClientFactory withUncaughtExceptionHandler(UncaughtExceptionHandler ue } - public ZooReader getZooReader() { + public ZooSession getZooSession() { ensureOpen(); - return zooReader; + return zooSession.get(); } protected long getTransportPoolMaxAgeMillis() { @@ -1110,7 +1095,14 @@ public synchronized ThriftTransportPool getTransportPool() { public synchronized ZookeeperLockChecker getTServerLockChecker() { ensureOpen(); if (this.zkLockChecker == null) { - this.zkLockChecker = new ZookeeperLockChecker(this); + // make this use its own ZooSession and ZooCache, because this is used by the + // tablet location cache, which is a static singleton reused by multiple clients + // so, it can't rely on being able to continue to use the same client's ZooCache, + // because that client could be closed, and its ZooSession also closed + // this needs to be fixed; TODO https://github.com/apache/accumulo/issues/2301 + var zk = info.getZooKeeperSupplier(ZookeeperLockChecker.class.getSimpleName()).get(); + this.zkLockChecker = + new ZookeeperLockChecker(new ZooCache(zk), getZooKeeperRoot() + Constants.ZTSERVERS); } return this.zkLockChecker; } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfo.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfo.java index 7306d8f8045..cebe40ec2ad 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfo.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfo.java @@ -20,10 +20,14 @@ import java.net.URL; import java.nio.file.Path; +import java.util.Optional; import java.util.Properties; +import java.util.function.Supplier; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; +import org.apache.accumulo.core.data.InstanceId; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.hadoop.conf.Configuration; /** @@ -38,6 +42,16 @@ public interface ClientInfo { */ String getInstanceName(); + /** + * @return Accumulo instanceId + */ + InstanceId getInstanceId(); + + /** + * @return a Supplier for creating new ZooKeeper client instances based on the configuration + */ + Supplier getZooKeeperSupplier(String clientName); + /** * @return Zookeeper connection information for Accumulo instance */ @@ -77,27 +91,27 @@ public interface ClientInfo { * @return ClientInfo given properties */ static ClientInfo from(Properties properties) { - return new ClientInfoImpl(properties); + return new ClientInfoImpl(properties, Optional.empty()); } /** * @return ClientInfo given URL path to client config file */ static ClientInfo from(URL propertiesURL) { - return new ClientInfoImpl(propertiesURL); + return new ClientInfoImpl(ClientInfoImpl.toProperties(propertiesURL), Optional.empty()); } /** * @return ClientInfo given properties and token */ static ClientInfo from(Properties properties, AuthenticationToken token) { - return new ClientInfoImpl(properties, token); + return new ClientInfoImpl(properties, Optional.of(token)); } /** * @return ClientInfo given path to client config file */ static ClientInfo from(Path propertiesFile) { - return new ClientInfoImpl(propertiesFile); + return new ClientInfoImpl(ClientInfoImpl.toProperties(propertiesFile), Optional.empty()); } } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfoImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfoImpl.java index 372906ca6a5..04911060c3f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfoImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfoImpl.java @@ -18,48 +18,70 @@ */ package org.apache.accumulo.core.clientImpl; +import static com.google.common.base.Suppliers.memoize; +import static java.util.Objects.requireNonNull; + import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.net.URL; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Optional; import java.util.Properties; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.ConfigurationTypeHelper; +import org.apache.accumulo.core.data.InstanceId; +import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.hadoop.conf.Configuration; +import com.google.common.base.Suppliers; + import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; public class ClientInfoImpl implements ClientInfo { private final Properties properties; - private AuthenticationToken token; - private final Configuration hadoopConf; - - public ClientInfoImpl(Path propertiesFile) { - this(ClientInfoImpl.toProperties(propertiesFile)); - } - public ClientInfoImpl(URL propertiesURL) { - this(ClientInfoImpl.toProperties(propertiesURL)); + // suppliers for lazily loading + private final Supplier tokenSupplier; + private final Supplier hadoopConf; + private final Supplier instanceId; + private final Function zooSessionForName; + + public ClientInfoImpl(Properties properties, Optional tokenOpt) { + this.properties = requireNonNull(properties); + // convert the optional to a supplier to delay retrieval from the properties unless needed + this.tokenSupplier = requireNonNull(tokenOpt).map(Suppliers::ofInstance) + .orElse(memoize(() -> ClientProperty.getAuthenticationToken(properties))); + this.hadoopConf = memoize(Configuration::new); + this.zooSessionForName = + name -> new ZooSession(name, getZooKeepers(), getZooKeepersSessionTimeOut(), null); + this.instanceId = memoize(() -> { + try (var zk = getZooKeeperSupplier(getClass().getSimpleName() + ".getInstanceName()").get()) { + return ZooUtil.getInstanceId(zk, getInstanceName()); + } + }); } - public ClientInfoImpl(Properties properties) { - this(properties, null); + @Override + public String getInstanceName() { + return getString(ClientProperty.INSTANCE_NAME); } - public ClientInfoImpl(Properties properties, AuthenticationToken token) { - this.properties = properties; - this.token = token; - this.hadoopConf = new Configuration(); + @Override + public InstanceId getInstanceId() { + return instanceId.get(); } @Override - public String getInstanceName() { - return getString(ClientProperty.INSTANCE_NAME); + public Supplier getZooKeeperSupplier(String clientName) { + return () -> zooSessionForName.apply(clientName); } @Override @@ -87,10 +109,7 @@ public Properties getProperties() { @Override public AuthenticationToken getAuthenticationToken() { - if (token == null) { - token = ClientProperty.getAuthenticationToken(properties); - } - return token; + return tokenSupplier.get(); } @Override @@ -134,6 +153,6 @@ public static Properties toProperties(URL propertiesURL) { @Override public Configuration getHadoopConf() { - return this.hadoopConf; + return hadoopConf.get(); } } diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java index 801b71f3d7a..e2ae20fd84a 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java @@ -394,22 +394,6 @@ public void waitForBalance() throws AccumuloException { } - /** - * Given a zooCache and instanceId, look up the instance name. - */ - public static String lookupInstanceName(ZooCache zooCache, InstanceId instanceId) { - checkArgument(zooCache != null, "zooCache is null"); - checkArgument(instanceId != null, "instanceId is null"); - for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) { - var bytes = zooCache.get(Constants.ZROOT + Constants.ZINSTANCES + "/" + name); - InstanceId iid = InstanceId.of(new String(bytes, UTF_8)); - if (iid.equals(instanceId)) { - return name; - } - } - return null; - } - @Override public InstanceId getInstanceId() { return context.getInstanceID(); diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java index 3fe472fdcf7..93231edd95f 100644 --- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java +++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java @@ -18,7 +18,8 @@ */ package org.apache.accumulo.core.clientImpl; -import org.apache.accumulo.core.Constants; +import static java.util.Objects.requireNonNull; + import org.apache.accumulo.core.clientImpl.TabletLocatorImpl.TabletServerLockChecker; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.lock.ServiceLock; @@ -28,9 +29,9 @@ public class ZookeeperLockChecker implements TabletServerLockChecker { private final ZooCache zc; private final String root; - ZookeeperLockChecker(ClientContext context) { - zc = context.getZooCache(); - this.root = context.getZooKeeperRoot() + Constants.ZTSERVERS; + ZookeeperLockChecker(ZooCache zooCache, String basePath) { + this.zc = requireNonNull(zooCache); + this.root = requireNonNull(basePath); } public boolean doesTabletServerLockExist(String server) { diff --git a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java index 80b21fc04d0..4884b1537a5 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/AdminUtil.java @@ -37,12 +37,11 @@ import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus; import org.apache.accumulo.core.fate.zookeeper.FateLock; import org.apache.accumulo.core.fate.zookeeper.FateLock.FateLockPath; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath; import org.apache.accumulo.core.util.FastFormat; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -211,7 +210,7 @@ public Map> getDanglingWaitingLocks() { /** * Returns a list of the FATE transactions, optionally filtered by transaction id and status. This * method does not process lock information, if lock information is desired, use - * {@link #getStatus(ReadOnlyTStore, ZooReader, ServiceLockPath, Set, EnumSet)} + * {@link #getStatus(ReadOnlyTStore, ZooSession, ServiceLockPath, Set, EnumSet)} * * @param zs read-only zoostore * @param filterTxid filter results to include for provided transaction ids. @@ -240,7 +239,7 @@ public List getTransactionStatus(ReadOnlyTStore zs, Set zs, ZooReader zk, + public FateStatus getStatus(ReadOnlyTStore zs, ZooSession zk, ServiceLock.ServiceLockPath lockPath, Set filterTxid, EnumSet filterStatus) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); @@ -254,19 +253,21 @@ public FateStatus getStatus(ReadOnlyTStore zs, ZooReader zk, /** * Walk through the lock nodes in zookeeper to find and populate held locks and waiting locks. * - * @param zk zookeeper reader + * @param zk zookeeper client * @param lockPath the zookeeper path for locks * @param heldLocks map for returning transactions with held locks * @param waitingLocks map for returning transactions with waiting locks * @throws KeeperException if initial lock list cannot be read. * @throws InterruptedException if thread interrupt detected while processing. */ - private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, + private void findLocks(ZooSession zk, final ServiceLock.ServiceLockPath lockPath, final Map> heldLocks, final Map> waitingLocks) throws KeeperException, InterruptedException { + var zr = zk.asReader(); + // stop with exception if lock ids cannot be retrieved from zookeeper - List lockedIds = zk.getChildren(lockPath.toString()); + List lockedIds = zr.getChildren(lockPath.toString()); for (String id : lockedIds) { @@ -274,14 +275,14 @@ private void findLocks(ZooReader zk, final ServiceLock.ServiceLockPath lockPath, FateLockPath fLockPath = FateLock.path(lockPath + "/" + id); List lockNodes = - FateLock.validateAndSort(fLockPath, zk.getChildren(fLockPath.toString())); + FateLock.validateAndSort(fLockPath, zr.getChildren(fLockPath.toString())); int pos = 0; boolean sawWriteLock = false; for (String node : lockNodes) { try { - byte[] data = zk.getData(lockPath + "/" + id + "/" + node); + byte[] data = zr.getData(lockPath + "/" + id + "/" + node); String[] lda = new String(data, UTF_8).split(":"); if (lda[0].charAt(0) == 'W') { @@ -407,12 +408,12 @@ private static boolean includeByTxid(Long tid, Set filterTxid) { return (filterTxid == null) || filterTxid.isEmpty() || filterTxid.contains(tid); } - public void printAll(ReadOnlyTStore zs, ZooReader zk, + public void printAll(ReadOnlyTStore zs, ZooSession zk, ServiceLock.ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException { print(zs, zk, tableLocksPath, new Formatter(System.out), null, null); } - public void print(ReadOnlyTStore zs, ZooReader zk, ServiceLock.ServiceLockPath tableLocksPath, + public void print(ReadOnlyTStore zs, ZooSession zk, ServiceLock.ServiceLockPath tableLocksPath, Formatter fmt, Set filterTxid, EnumSet filterStatus) throws KeeperException, InterruptedException { FateStatus fateStatus = getStatus(zs, zk, tableLocksPath, filterTxid, filterStatus); @@ -426,8 +427,7 @@ public void print(ReadOnlyTStore zs, ZooReader zk, ServiceLock.ServiceLockPat fmt.format(" %s transactions", fateStatus.getTransactions().size()); } - public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path, - String txidStr) { + public boolean prepDelete(TStore zs, ZooSession zk, ServiceLockPath path, String txidStr) { if (!checkGlobalLock(zk, path)) { return false; } @@ -463,7 +463,7 @@ public boolean prepDelete(TStore zs, ZooReaderWriter zk, ServiceLockPath path return state; } - public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, + public boolean prepFail(TStore zs, ZooSession zk, ServiceLockPath zLockManagerPath, String txidStr) { if (!checkGlobalLock(zk, zLockManagerPath)) { return false; @@ -507,19 +507,21 @@ public boolean prepFail(TStore zs, ZooReaderWriter zk, ServiceLockPath zLockM return state; } - public void deleteLocks(ZooReaderWriter zk, ServiceLock.ServiceLockPath path, String txidStr) + public void deleteLocks(ZooSession zk, ServiceLock.ServiceLockPath path, String txidStr) throws KeeperException, InterruptedException { + var zrw = zk.asReaderWriter(); + // delete any locks assoc w/ fate operation - List lockedIds = zk.getChildren(path.toString()); + List lockedIds = zrw.getChildren(path.toString()); for (String id : lockedIds) { - List lockNodes = zk.getChildren(path + "/" + id); + List lockNodes = zrw.getChildren(path + "/" + id); for (String node : lockNodes) { String lockPath = path + "/" + id + "/" + node; - byte[] data = zk.getData(path + "/" + id + "/" + node); + byte[] data = zrw.getData(path + "/" + id + "/" + node); String[] lda = new String(data, UTF_8).split(":"); if (lda[1].equals(txidStr)) { - zk.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); } } } @@ -528,9 +530,9 @@ public void deleteLocks(ZooReaderWriter zk, ServiceLock.ServiceLockPath path, St @SuppressFBWarnings(value = "DM_EXIT", justification = "TODO - should probably avoid System.exit here; " + "this code is used by the fate admin shell command") - public boolean checkGlobalLock(ZooReaderWriter zk, ServiceLockPath zLockManagerPath) { + public boolean checkGlobalLock(ZooSession zk, ServiceLockPath zLockManagerPath) { try { - if (ServiceLock.getLockData(zk.getZooKeeper(), zLockManagerPath) != null) { + if (ServiceLock.getLockData(zk, zLockManagerPath) != null) { System.err.println("ERROR: Manager lock is held, not running"); if (this.exitOnError) { System.exit(1); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java index ac4bf7f8769..7421f2e2693 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java @@ -45,6 +45,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.util.CountDownTimer; import org.apache.accumulo.core.util.FastFormat; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; @@ -61,7 +62,8 @@ public class ZooStore implements TStore { private static final Logger log = LoggerFactory.getLogger(ZooStore.class); private String path; - private ZooReaderWriter zk; + private ZooSession zk; + private ZooReaderWriter zrw; private String lastReserved = ""; private Set reserved; private Map deferred; @@ -101,14 +103,15 @@ private long parseTid(String txdir) { return Long.parseLong(txdir.split("_")[1], 16); } - public ZooStore(String path, ZooReaderWriter zk) throws KeeperException, InterruptedException { + public ZooStore(String path, ZooSession zk) throws KeeperException, InterruptedException { this.path = path; this.zk = zk; + this.zrw = zk.asReaderWriter(); this.reserved = new HashSet<>(); this.deferred = new HashMap<>(); - zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); + this.zrw.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); } /** @@ -122,7 +125,7 @@ public long create() { try { // looking at the code for SecureRandom, it appears to be thread safe long tid = RANDOM.get().nextLong() & 0x7fffffffffffffffL; - zk.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8), + zrw.putPersistentData(getTXPath(tid), TStatus.NEW.name().getBytes(UTF_8), NodeExistsPolicy.FAIL); return tid; } catch (NodeExistsException nee) { @@ -143,7 +146,7 @@ public long reserve() { events = statusChangeEvents; } - List txdirs = new ArrayList<>(zk.getChildren(path)); + List txdirs = new ArrayList<>(zrw.getChildren(path)); Collections.sort(txdirs); synchronized (this) { @@ -181,7 +184,7 @@ public long reserve() { // have reserved id, status should not change try { - TStatus status = TStatus.valueOf(new String(zk.getData(path + "/" + txdir), UTF_8)); + TStatus status = TStatus.valueOf(new String(zrw.getData(path + "/" + txdir), UTF_8)); if (status == TStatus.SUBMITTED || status == TStatus.IN_PROGRESS || status == TStatus.FAILED_IN_PROGRESS) { return tid; @@ -320,7 +323,7 @@ public Repo top(long tid) { throw new IllegalStateException(ex); } - byte[] ser = zk.getData(txpath + "/" + top); + byte[] ser = zrw.getData(txpath + "/" + top); @SuppressWarnings("unchecked") var deserialized = (Repo) deserialize(ser); return deserialized; @@ -336,7 +339,7 @@ public Repo top(long tid) { } private String findTop(String txpath) throws KeeperException, InterruptedException { - List ops = zk.getChildren(txpath); + List ops = zrw.getChildren(txpath); ops = new ArrayList<>(ops); @@ -366,7 +369,7 @@ public void push(long tid, Repo repo) throws StackOverflowException { throw new StackOverflowException("Repo stack size too large"); } - zk.putPersistentSequential(txpath + "/repo_", serialize(repo)); + zrw.putPersistentSequential(txpath + "/repo_", serialize(repo)); } catch (StackOverflowException soe) { throw soe; } catch (KeeperException | InterruptedException e) { @@ -384,7 +387,7 @@ public void pop(long tid) { if (top == null) { throw new IllegalStateException("Tried to pop when empty " + FateTxId.formatTid(tid)); } - zk.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(txpath + "/" + top, NodeMissingPolicy.SKIP); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -392,7 +395,7 @@ public void pop(long tid) { private TStatus _getStatus(long tid) { try { - return TStatus.valueOf(new String(zk.getData(getTXPath(tid)), UTF_8)); + return TStatus.valueOf(new String(zrw.getData(getTXPath(tid)), UTF_8)); } catch (NoNodeException nne) { return TStatus.UNKNOWN; } catch (KeeperException | InterruptedException e) { @@ -437,7 +440,7 @@ public void setStatus(long tid, TStatus status) { verifyReserved(tid); try { - zk.putPersistentData(getTXPath(tid), status.name().getBytes(UTF_8), + zrw.putPersistentData(getTXPath(tid), status.name().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); @@ -454,7 +457,7 @@ public void delete(long tid) { verifyReserved(tid); try { - zk.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); + zrw.recursiveDelete(getTXPath(tid), NodeMissingPolicy.SKIP); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -466,7 +469,7 @@ public void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable so) { try { if (so instanceof String) { - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), + zrw.putPersistentData(getTXPath(tid) + "/" + txInfo, ("S " + so).getBytes(UTF_8), NodeExistsPolicy.OVERWRITE); } else { byte[] sera = serialize(so); @@ -474,7 +477,7 @@ public void setTransactionInfo(long tid, Fate.TxInfo txInfo, Serializable so) { System.arraycopy(sera, 0, data, 2, sera.length); data[0] = 'O'; data[1] = ' '; - zk.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); + zrw.putPersistentData(getTXPath(tid) + "/" + txInfo, data, NodeExistsPolicy.OVERWRITE); } } catch (KeeperException | InterruptedException e2) { throw new IllegalStateException(e2); @@ -486,7 +489,7 @@ public Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo) { verifyReserved(tid); try { - byte[] data = zk.getData(getTXPath(tid) + "/" + txInfo); + byte[] data = zrw.getData(getTXPath(tid) + "/" + txInfo); if (data[0] == 'O') { byte[] sera = new byte[data.length - 2]; @@ -508,7 +511,7 @@ public Serializable getTransactionInfo(long tid, Fate.TxInfo txInfo) { public List list() { try { ArrayList l = new ArrayList<>(); - List transactions = zk.getChildren(path); + List transactions = zrw.getChildren(path); for (String txid : transactions) { l.add(parseTid(txid)); } @@ -523,7 +526,7 @@ public long timeCreated(long tid) { verifyReserved(tid); try { - Stat stat = zk.getZooKeeper().exists(getTXPath(tid), false); + Stat stat = zk.exists(getTXPath(tid), null); return stat.getCtime(); } catch (Exception e) { return 0; @@ -537,7 +540,7 @@ public List> getStack(long tid) { outer: while (true) { List ops; try { - ops = zk.getChildren(txpath); + ops = zrw.getChildren(txpath); } catch (KeeperException.NoNodeException e) { return Collections.emptyList(); } catch (KeeperException | InterruptedException e1) { @@ -553,7 +556,7 @@ public List> getStack(long tid) { if (child.startsWith("repo_")) { byte[] ser; try { - ser = zk.getData(txpath + "/" + child); + ser = zrw.getData(txpath + "/" + child); @SuppressWarnings("unchecked") var repo = (ReadOnlyRepo) deserialize(ser); dops.add(repo); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java index fc2b6f1c414..e24448585e2 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; import static org.apache.accumulo.core.util.LazySingletons.RANDOM; import java.util.Collections; @@ -31,15 +32,16 @@ import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Consumer; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath; import org.apache.accumulo.core.lock.ServiceLockData; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,10 +53,13 @@ * A cache for values stored in ZooKeeper. Values are kept up to date as they change. */ public class ZooCache { + + public interface ZooCacheWatcher extends Consumer {} + private static final Logger log = LoggerFactory.getLogger(ZooCache.class); private final ZCacheWatcher watcher = new ZCacheWatcher(); - private final Watcher externalWatcher; + private final Optional externalWatcher; private final ReadWriteLock cacheLock = new ReentrantReadWriteLock(false); private final Lock cacheWriteLock = cacheLock.writeLock(); @@ -64,7 +69,7 @@ public class ZooCache { private final HashMap statCache; private final HashMap> childrenCache; - private final ZooReader zReader; + private final ZooSession zk; private volatile boolean closed = false; @@ -139,19 +144,6 @@ private static class ImmutableCacheCopies { private volatile ImmutableCacheCopies immutableCache = new ImmutableCacheCopies(0); private long updateCount = 0; - /** - * Returns a ZooKeeper session. Calls should be made within run of ZooRunnable after caches are - * checked. This will be performed at each retry of the run method. Calls to this method should be - * made, ideally, after cache checks since other threads may have succeeded when updating the - * cache. Doing this will ensure that we don't pay the cost of retrieving a ZooKeeper session on - * each retry until we've ensured the caches aren't populated for a given node. - * - * @return ZooKeeper session. - */ - private ZooKeeper getZooKeeper() { - return zReader.getZooKeeper(); - } - private class ZCacheWatcher implements Watcher { @Override public void process(WatchedEvent event) { @@ -196,20 +188,33 @@ public void process(WatchedEvent event) { break; } - if (externalWatcher != null) { - externalWatcher.process(event); - } + externalWatcher.ifPresent(w -> w.accept(event)); } } + /** + * Creates a new cache without an external watcher. + * + * @param zk the ZooKeeper instance + * @throws NullPointerException if zk is {@code null} + */ + public ZooCache(ZooSession zk) { + this(zk, Optional.empty()); + } + /** * Creates a new cache. The given watcher is called whenever a watched node changes. * - * @param reader ZooKeeper reader + * @param zk the ZooKeeper instance * @param watcher watcher object + * @throws NullPointerException if zk or watcher is {@code null} */ - public ZooCache(ZooReader reader, Watcher watcher) { - this.zReader = reader; + public ZooCache(ZooSession zk, ZooCacheWatcher watcher) { + this(zk, Optional.of(watcher)); + } + + private ZooCache(ZooSession zk, Optional watcher) { + this.zk = requireNonNull(zk); this.cache = new HashMap<>(); this.statCache = new HashMap<>(); this.childrenCache = new HashMap<>(); @@ -305,9 +310,7 @@ public List run() throws KeeperException, InterruptedException { return childrenCache.get(zPath); } - final ZooKeeper zooKeeper = getZooKeeper(); - - List children = zooKeeper.getChildren(zPath, watcher); + List children = zk.getChildren(zPath, watcher); if (children != null) { children = List.copyOf(children); } @@ -377,8 +380,7 @@ public byte[] run() throws KeeperException, InterruptedException { */ cacheWriteLock.lock(); try { - final ZooKeeper zooKeeper = getZooKeeper(); - Stat stat = zooKeeper.exists(zPath, watcher); + Stat stat = zk.exists(zPath, watcher); byte[] data = null; if (stat == null) { if (log.isTraceEnabled()) { @@ -386,7 +388,7 @@ public byte[] run() throws KeeperException, InterruptedException { } } else { try { - data = zooKeeper.getData(zPath, watcher, stat); + data = zk.getData(zPath, watcher, stat); zstat = new ZcStat(stat); } catch (KeeperException.BadVersionException | KeeperException.NoNodeException e1) { throw new ConcurrentModificationException(); diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java deleted file mode 100644 index 2e1987af344..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactory.java +++ /dev/null @@ -1,128 +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.fate.zookeeper; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.accumulo.core.singletons.SingletonManager; -import org.apache.accumulo.core.singletons.SingletonService; - -/** - * A factory for {@link ZooCache} instances. - *

- * Implementation note: We were using the instances map to track all the instances that have been - * created, so we could explicitly close them when the SingletonManager detected that the last - * legacy client (using Connector/ZooKeeperInstance) has gone away. This class may no longer be - * needed, since the legacy client code has been removed, so long as the ZooCache instances it is - * tracking are managed as resources within ClientContext or ServerContext, and explicitly closed - * when those are closed. - */ -public class ZooCacheFactory { - - private static final Map instances = new HashMap<>(); - private static boolean enabled = true; - - public ZooCacheFactory() {} - - private static boolean isEnabled() { - synchronized (instances) { - return enabled; - } - } - - private static void enable() { - synchronized (instances) { - enabled = true; - } - } - - private static void disable() { - synchronized (instances) { - try { - instances.values().forEach(ZooCache::close); - } finally { - instances.clear(); - enabled = false; - } - } - } - - static { - // important because of ZOOKEEPER-2368.. when zookeeper client is closed it does not generate an - // event! - SingletonManager.register(new SingletonService() { - - @Override - public synchronized boolean isEnabled() { - return ZooCacheFactory.isEnabled(); - } - - @Override - public synchronized void enable() { - ZooCacheFactory.enable(); - } - - @Override - public synchronized void disable() { - ZooCacheFactory.disable(); - } - }); - - } - - /** - * Gets a {@link ZooCache}. The same object may be returned for multiple calls with the same - * arguments. - * - * @param zooKeepers comma-separated list of ZooKeeper host[:port]s - * @param sessionTimeout session timeout - * @return cache object - */ - public ZooCache getZooCache(String zooKeepers, int sessionTimeout) { - String key = zooKeepers + ":" + sessionTimeout; - synchronized (instances) { - if (!isEnabled()) { - throw new IllegalStateException("The Accumulo singleton for zookeeper caching is " - + "disabled. This is likely caused by all AccumuloClients being closed"); - } - return instances.computeIfAbsent(key, k -> getNewZooCache(zooKeepers, sessionTimeout)); - } - } - - /** - * Always return a new {@link ZooCache}. - * - * @param zooKeepers comma-separated list of ZooKeeper host[:port]s - * @param sessionTimeout session timeout - * @return a new instance - */ - public ZooCache getNewZooCache(String zooKeepers, int sessionTimeout) { - return new ZooCache(new ZooReader(zooKeepers, sessionTimeout), null); - } - - /** - * Resets the factory. All cached objects are flushed. - */ - void reset() { - synchronized (instances) { - instances.clear(); - } - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java index 9d9738de1bf..7f67241b4a2 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java @@ -29,10 +29,10 @@ import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.Retry.RetryFactory; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,36 +45,26 @@ public class ZooReader { .incrementBy(Duration.ofMillis(250)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5) .logInterval(Duration.ofMinutes(3)).createFactory(); - protected final String connectString; - protected final int timeout; + private final ZooSession zk; - public ZooReader(String connectString, int timeout) { - this.connectString = requireNonNull(connectString); - this.timeout = timeout; - } - - public ZooReaderWriter asWriter(String secret) { - return new ZooReaderWriter(connectString, timeout, secret); + /** + * Decorate a ZooKeeper with additional, more convenient functionality. + * + * @param zk the ZooKeeper instance + * @throws NullPointerException if zk is {@code null} + */ + public ZooReader(ZooSession zk) { + this.zk = requireNonNull(zk); } - protected ZooKeeper getZooKeeper() { - return ZooSession.getAnonymousSession(connectString, timeout); + protected ZooSession getZooKeeper() { + return zk; } protected RetryFactory getRetryFactory() { return RETRY_FACTORY; } - /** - * Returns the requested ZooKeeper client session timeout. The client may negotiate a different - * value and the actual negotiated value may change after a re-connect. - * - * @return the timeout in milliseconds - */ - public int getSessionTimeout() { - return timeout; - } - public byte[] getData(String zPath) throws KeeperException, InterruptedException { return retryLoop(zk -> zk.getData(zPath, null, null)); } @@ -135,11 +125,11 @@ public void sync(final String path) throws KeeperException, InterruptedException } protected interface ZKFunction { - R apply(ZooKeeper zk) throws KeeperException, InterruptedException; + R apply(ZooSession zk) throws KeeperException, InterruptedException; } protected interface ZKFunctionMutator { - R apply(ZooKeeper zk) + R apply(ZooSession zk) throws KeeperException, InterruptedException, AcceptableThriftTableOperationException; } diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriter.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriter.java index ed0566bbba8..2f105262253 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriter.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriter.java @@ -18,20 +18,17 @@ */ package org.apache.accumulo.core.fate.zookeeper; -import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; import java.util.List; import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; @@ -40,32 +37,14 @@ public interface Mutator { byte[] mutate(byte[] currentValue) throws AcceptableThriftTableOperationException; } - public ZooReaderWriter(AccumuloConfiguration conf) { - this(conf.get(Property.INSTANCE_ZK_HOST), - (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), - conf.get(Property.INSTANCE_SECRET)); - } - - private final String secret; - private final byte[] auth; - - ZooReaderWriter(String connectString, int timeoutInMillis, String secret) { - super(connectString, timeoutInMillis); - this.secret = requireNonNull(secret); - this.auth = ("accumulo:" + secret).getBytes(UTF_8); - } - - @Override - public ZooReaderWriter asWriter(String secret) { - if (this.secret.equals(secret)) { - return this; - } - return super.asWriter(secret); - } - - @Override - public ZooKeeper getZooKeeper() { - return ZooSession.getAuthenticatedSession(connectString, timeout, "digest", auth); + /** + * Decorate a ZooKeeper with additional, more convenient functionality. + * + * @param zk the ZooKeeper instance + * @throws NullPointerException if zk is {@code null} + */ + public ZooReaderWriter(ZooSession zk) { + super(zk); } /** diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java deleted file mode 100644 index d3cee2bb8b4..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooSession.java +++ /dev/null @@ -1,252 +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.fate.zookeeper; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; - -import java.io.IOException; -import java.net.UnknownHostException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.accumulo.core.singletons.SingletonManager; -import org.apache.accumulo.core.singletons.SingletonService; -import org.apache.accumulo.core.util.AddressUtil; -import org.apache.accumulo.core.util.UtilWaitThread; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.ZooKeeper.States; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ZooSession { - - public static class ZooSessionShutdownException extends RuntimeException { - - public ZooSessionShutdownException(String msg) { - super(msg); - } - - private static final long serialVersionUID = 1L; - - } - - private static final Logger log = LoggerFactory.getLogger(ZooSession.class); - - private static class ZooSessionInfo { - public ZooSessionInfo(ZooKeeper zooKeeper) { - this.zooKeeper = zooKeeper; - } - - ZooKeeper zooKeeper; - } - - private static Map sessions = new HashMap<>(); - - static { - SingletonManager.register(new SingletonService() { - - @Override - public boolean isEnabled() { - return ZooSession.isEnabled(); - } - - @Override - public void enable() { - ZooSession.enable(); - } - - @Override - public void disable() { - ZooSession.disable(); - } - }); - } - - private static String sessionKey(String keepers, int timeout, String scheme, byte[] auth) { - return keepers + ":" + timeout + ":" + (scheme == null ? "" : scheme) + ":" - + (auth == null ? "" : new String(auth, UTF_8)); - } - - private static class ZooWatcher implements Watcher { - - @Override - public void process(WatchedEvent event) { - if (event.getState() == KeeperState.Expired) { - log.debug("Session expired; {}", event); - } - } - - } - - /** - * @param connectString in the form of host1:port1,host2:port2/chroot/path - * @param timeout in milliseconds - * @param scheme authentication type, e.g. 'digest', may be null - * @param auth authentication-scheme-specific token, may be null - * @param watcher ZK notifications, may be null - */ - static ZooKeeper connect(String connectString, int timeout, String scheme, byte[] auth, - Watcher watcher) { - final int TIME_BETWEEN_CONNECT_CHECKS_MS = 100; - int connectTimeWait = Math.min(10_000, timeout); - boolean tryAgain = true; - long sleepTime = 100; - ZooKeeper zooKeeper = null; - - long startTime = System.nanoTime(); - - while (tryAgain) { - try { - zooKeeper = new ZooKeeper(connectString, timeout, watcher); - // it may take some time to get connected to zookeeper if some of the servers are down - for (int i = 0; i < connectTimeWait / TIME_BETWEEN_CONNECT_CHECKS_MS && tryAgain; i++) { - if (zooKeeper.getState().equals(States.CONNECTED)) { - if (auth != null) { - ZooUtil.auth(zooKeeper, scheme, auth); - } - tryAgain = false; - } else { - UtilWaitThread.sleep(TIME_BETWEEN_CONNECT_CHECKS_MS); - } - } - - } catch (IOException e) { - if (e instanceof UnknownHostException) { - /* - * Make sure we wait at least as long as the JVM TTL for negative DNS responses - */ - int ttl = AddressUtil.getAddressCacheNegativeTtl((UnknownHostException) e); - sleepTime = Math.max(sleepTime, (ttl + 1) * 1000L); - } - log.warn("Connection to zooKeeper failed, will try again in " - + String.format("%.2f secs", sleepTime / 1000.0), e); - } finally { - if (tryAgain && zooKeeper != null) { - try { - zooKeeper.close(); - zooKeeper = null; - } catch (InterruptedException e) { - log.warn("interrupted", e); - } - } - } - - long stopTime = System.nanoTime(); - long duration = NANOSECONDS.toMillis(stopTime - startTime); - - if (duration > 2L * timeout) { - throw new IllegalStateException("Failed to connect to zookeeper (" + connectString - + ") within 2x zookeeper timeout period " + timeout); - } - - if (tryAgain) { - if (2L * timeout < duration + sleepTime + connectTimeWait) { - sleepTime = 2L * timeout - duration - connectTimeWait; - } - if (sleepTime < 0) { - connectTimeWait -= sleepTime; - sleepTime = 0; - } - UtilWaitThread.sleep(sleepTime); - if (sleepTime < 10000) { - sleepTime = sleepTime + (long) (sleepTime * RANDOM.get().nextDouble()); - } - } - } - - return zooKeeper; - } - - public static ZooKeeper getAuthenticatedSession(String connectString, int timeout, String scheme, - byte[] auth) { - return getSession(connectString, timeout, scheme, auth); - } - - public static ZooKeeper getAnonymousSession(String connectString, int timeout) { - return getSession(connectString, timeout, null, null); - } - - private static synchronized ZooKeeper getSession(String connectString, int timeout, String scheme, - byte[] auth) { - - if (sessions == null) { - throw new ZooSessionShutdownException( - "The Accumulo singleton that that tracks zookeeper session is disabled. This is likely " - + "caused by all AccumuloClients being closed or garbage collected."); - } - - String sessionKey = sessionKey(connectString, timeout, scheme, auth); - - // a read-only session can use a session with authorizations, so cache a copy for it w/out auths - String readOnlySessionKey = sessionKey(connectString, timeout, null, null); - ZooSessionInfo zsi = sessions.get(sessionKey); - if (zsi != null && zsi.zooKeeper.getState() == States.CLOSED) { - log.debug("Removing closed ZooKeeper session to {}", connectString); - if (auth != null && sessions.get(readOnlySessionKey) == zsi) { - sessions.remove(readOnlySessionKey); - } - zsi = null; - sessions.remove(sessionKey); - } - - if (zsi == null) { - ZooWatcher watcher = new ZooWatcher(); - log.debug("Connecting to {} with timeout {} with auth", connectString, timeout); - zsi = new ZooSessionInfo(connect(connectString, timeout, scheme, auth, watcher)); - sessions.put(sessionKey, zsi); - if (auth != null && !sessions.containsKey(readOnlySessionKey)) { - sessions.put(readOnlySessionKey, zsi); - } - } - return zsi.zooKeeper; - } - - private static synchronized boolean isEnabled() { - return sessions != null; - } - - private static synchronized void enable() { - if (sessions != null) { - return; - } - - sessions = new HashMap<>(); - } - - private static synchronized void disable() { - if (sessions == null) { - return; - } - - for (ZooSessionInfo zsi : sessions.values()) { - try { - zsi.zooKeeper.close(); - } catch (Exception e) { - log.debug("Error closing zookeeper during shutdown", e); - } - } - - sessions = null; - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java index 47d906fedb4..9bdbca36209 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooUtil.java @@ -19,31 +19,38 @@ package org.apache.accumulo.core.fate.zookeeper; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; import java.math.BigInteger; -import java.security.NoSuchAlgorithmException; import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.TreeMap; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.InstanceId; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; -import org.apache.zookeeper.server.auth.DigestAuthenticationProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ZooUtil { + private static final Logger log = LoggerFactory.getLogger(ZooUtil.class); + + private ZooUtil() {} + public enum NodeExistsPolicy { SKIP, OVERWRITE, FAIL } @@ -57,9 +64,9 @@ public enum NodeMissingPolicy { DateTimeFormatter.ofPattern("EEE MMM dd HH:mm:ss 'UTC' yyyy"); public static class LockID { - public long eid; - public String path; - public String node; + public final long eid; + public final String path; + public final String node; public LockID(String root, String serializedLID) { String[] sa = serializedLID.split("\\$"); @@ -85,7 +92,6 @@ public LockID(String path, String node, long eid) { } public String serialize(String root) { - return path.substring(root.length()) + "/" + node + "$" + Long.toHexString(eid); } @@ -116,7 +122,7 @@ public static String getRoot(final InstanceId instanceId) { /** * This method will delete a node and all its children. */ - public static void recursiveDelete(ZooKeeper zooKeeper, String zPath, NodeMissingPolicy policy) + public static void recursiveDelete(ZooSession zooKeeper, String zPath, NodeMissingPolicy policy) throws KeeperException, InterruptedException { if (policy == NodeMissingPolicy.CREATE) { throw new IllegalArgumentException(policy.name() + " is invalid for this operation"); @@ -169,27 +175,76 @@ private static String getFmtTime(final long epoch) { return fmt.format(timestamp); } - /** - * Get the ZooKeeper digest based on the instance secret that is used within ZooKeeper for - * authentication. This method is primary intended to be used to validate ZooKeeper ACLs. Use - * {@link #digestAuth(ZooKeeper, String)} to add authorizations to ZooKeeper. - */ - public static Id getZkDigestAuthId(final String secret) { + public static String getInstanceName(ZooSession zk, InstanceId instanceId) { + requireNonNull(zk); + var instanceIdBytes = requireNonNull(instanceId).canonical().getBytes(UTF_8); + for (String name : getInstanceNames(zk)) { + var bytes = getInstanceIdBytesFromName(zk, name); + if (Arrays.equals(bytes, instanceIdBytes)) { + return name; + } + } + return null; + } + + private static List getInstanceNames(ZooSession zk) { try { - final String scheme = "digest"; - String auth = DigestAuthenticationProvider.generateDigest("accumulo:" + secret); - return new Id(scheme, auth); - } catch (NoSuchAlgorithmException ex) { - throw new IllegalArgumentException("Could not generate ZooKeeper digest string", ex); + return zk.asReader().getChildren(Constants.ZROOT + Constants.ZINSTANCES); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Interrupted reading instance names from ZooKeeper", e); + } catch (KeeperException e) { + throw new IllegalStateException("Failed to read instance names from ZooKeeper", e); } } - public static void digestAuth(ZooKeeper zoo, String secret) { - auth(zoo, "digest", ("accumulo:" + secret).getBytes(UTF_8)); + private static byte[] getInstanceIdBytesFromName(ZooSession zk, String name) { + try { + return zk.asReader() + .getData(Constants.ZROOT + Constants.ZINSTANCES + "/" + requireNonNull(name)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException( + "Interrupted reading InstanceId from ZooKeeper for instance named " + name, e); + } catch (KeeperException e) { + log.warn("Failed to read InstanceId from ZooKeeper for instance named {}", name, e); + return null; + } + } + + public static Map getInstanceMap(ZooSession zk) { + Map idMap = new TreeMap<>(); + getInstanceNames(zk).forEach(name -> { + byte[] instanceId = getInstanceIdBytesFromName(zk, name); + if (instanceId != null) { + idMap.put(name, InstanceId.of(new String(instanceId, UTF_8))); + } + }); + return idMap; } - public static void auth(ZooKeeper zoo, String scheme, byte[] auth) { - zoo.addAuthInfo(scheme, auth); + public static InstanceId getInstanceId(ZooSession zk, String name) { + byte[] data = getInstanceIdBytesFromName(zk, name); + if (data == null) { + throw new IllegalStateException("Instance name " + name + " does not exist in ZooKeeper. " + + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list."); + } + String instanceIdString = new String(data, UTF_8); + try { + // verify that the instanceId found via the name actually exists + if (zk.asReader().getData(Constants.ZROOT + "/" + instanceIdString) == null) { + throw new IllegalStateException("InstanceId " + instanceIdString + + " pointed to by the name " + name + " does not exist in ZooKeeper"); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Interrupted verifying InstanceId " + instanceIdString + + " pointed to by instance named " + name + " actually exists in ZooKeeper", e); + } catch (KeeperException e) { + throw new IllegalStateException("Failed to verify InstanceId " + instanceIdString + + " pointed to by instance named " + name + " actually exists in ZooKeeper", e); + } + return InstanceId.of(instanceIdString); } } diff --git a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java index ced0cee2527..25cbc98cb6f 100644 --- a/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java +++ b/core/src/main/java/org/apache/accumulo/core/lock/ServiceLock.java @@ -32,6 +32,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; @@ -39,7 +40,6 @@ import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,7 +100,7 @@ public interface AccumuloLockWatcher extends LockWatcher { } private final ServiceLockPath path; - protected final ZooKeeper zooKeeper; + protected final ZooSession zooKeeper; private final Prefix vmLockPrefix; private LockWatcher lockWatcher; @@ -111,7 +111,7 @@ public interface AccumuloLockWatcher extends LockWatcher { private String createdNodeName; private String watchingNodeName; - public ServiceLock(ZooKeeper zookeeper, ServiceLockPath path, UUID uuid) { + public ServiceLock(ZooSession zookeeper, ServiceLockPath path, UUID uuid) { this.zooKeeper = requireNonNull(zookeeper); this.path = requireNonNull(path); try { @@ -657,7 +657,7 @@ public static boolean isLockHeld(ZooCache zc, LockID lid) { return zc.get(lid.path + "/" + lid.node, stat) != null && stat.getEphemeralOwner() == lid.eid; } - public static Optional getLockData(ZooKeeper zk, ServiceLockPath path) + public static Optional getLockData(ZooSession zk, ServiceLockPath path) throws KeeperException, InterruptedException { List children = validateAndSort(path, zk.getChildren(path.toString(), null)); @@ -668,7 +668,7 @@ public static Optional getLockData(ZooKeeper zk, ServiceLockPat String lockNode = children.get(0); - byte[] data = zk.getData(path + "/" + lockNode, false, null); + byte[] data = zk.getData(path + "/" + lockNode, null, null); if (data == null) { data = new byte[0]; } @@ -791,7 +791,7 @@ public boolean verifyLockAtSource() { return false; } try { - return null != this.zooKeeper.exists(lockPath, false); + return null != this.zooKeeper.exists(lockPath, null); } catch (KeeperException | InterruptedException | RuntimeException e) { LOG.error("Error verfiying lock at {}", lockPath, e); return false; diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java index 95045a24870..957eb9a86aa 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java @@ -589,7 +589,7 @@ private static TabletMetadata getRootMetadata(ClientContext ctx, case EVENTUAL: return getRootMetadata(zkRoot, ctx.getZooCache()); case IMMEDIATE: - ZooReader zooReader = ctx.getZooReader(); + ZooReader zooReader = ctx.getZooSession().asReader(); try { var path = zkRoot + RootTable.ZROOT_TABLET; // attempt (see ZOOKEEPER-1675) to ensure the latest root table metadata is read from diff --git a/core/src/main/java/org/apache/accumulo/core/util/MonitorUtil.java b/core/src/main/java/org/apache/accumulo/core/util/MonitorUtil.java index 6ea9d343759..87ed8448e3c 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/MonitorUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/MonitorUtil.java @@ -32,7 +32,7 @@ public class MonitorUtil { public static String getLocation(ClientContext context) throws KeeperException, InterruptedException { - return getLocation(context.getZooReader(), context); + return getLocation(context.getZooSession().asReader(), context); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java index c993cf4cb19..a491cf6b707 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java @@ -116,7 +116,7 @@ public static Map> getCompactorAddrs(ClientContext conte try { final Map> queuesAndAddresses = new HashMap<>(); final String compactorQueuesPath = context.getZooKeeperRoot() + Constants.ZCOMPACTORS; - ZooReader zooReader = context.getZooReader(); + ZooReader zooReader = context.getZooSession().asReader(); List queues = zooReader.getChildren(compactorQueuesPath); for (String queue : queues) { queuesAndAddresses.putIfAbsent(queue, new HashSet<>()); diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooSession.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooSession.java new file mode 100644 index 00000000000..032eecc4534 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooSession.java @@ -0,0 +1,312 @@ +/* + * 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.zookeeper; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.accumulo.core.util.LazySingletons.RANDOM; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.fate.zookeeper.ZooReader; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.util.AddressUtil; +import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.zookeeper.AsyncCallback.StringCallback; +import org.apache.zookeeper.AsyncCallback.VoidCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.Watcher.WatcherType; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A ZooKeeper client facade that maintains a ZooKeeper delegate instance. If the delegate instance + * loses its session, it is replaced with a new instance to establish a new session. Any Watchers + * registered on a session will need to monitor for the session expired event triggered from the old + * delegate, and must be reset on the new session if you intend them to monitor any further events. + * That is no different than if you created a new ZooKeeper instance directly after the first one + * expired. + */ +public class ZooSession implements AutoCloseable { + + public static class ZKUtil { + public static void deleteRecursive(ZooSession zk, final String pathRoot) + throws InterruptedException, KeeperException { + org.apache.zookeeper.ZKUtil.deleteRecursive(zk.verifyConnected(), pathRoot); + } + + public static void visitSubTreeDFS(ZooSession zk, final String path, boolean watch, + StringCallback cb) throws KeeperException, InterruptedException { + org.apache.zookeeper.ZKUtil.visitSubTreeDFS(zk.verifyConnected(), path, watch, cb); + } + } + + private static class ZooSessionWatcher implements Watcher { + + private final String connectionName; + private final AtomicReference lastState = new AtomicReference<>(null); + + public ZooSessionWatcher(String connectionName) { + this.connectionName = connectionName; + } + + @Override + public void process(WatchedEvent event) { + final var newState = event.getState(); + var oldState = lastState.getAndUpdate(s -> newState); + if (oldState == null) { + log.debug("{} state changed to {}", connectionName, newState); + } else if (newState != oldState) { + log.debug("{} state changed from {} to {}", connectionName, oldState, newState); + } + } + } + + private static final Logger log = LoggerFactory.getLogger(ZooSession.class); + + private static void closeZk(ZooKeeper zk) { + if (zk != null) { + try { + zk.close(); + } catch (InterruptedException e) { + // ZooKeeper doesn't actually throw this; it's just there for backwards compatibility + Thread.currentThread().interrupt(); + } + } + } + + private static void digestAuth(ZooKeeper zoo, String secret) { + zoo.addAuthInfo("digest", ("accumulo:" + requireNonNull(secret)).getBytes(UTF_8)); + } + + private final AtomicBoolean closed = new AtomicBoolean(); + private final AtomicLong connectCounter; + private final String connectString; + private final AtomicReference delegate = new AtomicReference<>(); + private final String instanceSecret; + private final String sessionName; + private final int timeout; + private final ZooReaderWriter zrw; + + /** + * Construct a new ZooKeeper client, retrying indefinitely if it doesn't work right away. The + * caller is responsible for closing instances returned from this method. + * + * @param clientName a convenient name for logging its connection state changes + * @param conf a convenient carrier of ZK connection information using Accumulo properties + */ + public ZooSession(String clientName, AccumuloConfiguration conf) { + this(clientName, conf.get(Property.INSTANCE_ZK_HOST), + (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), + conf.get(Property.INSTANCE_SECRET)); + } + + /** + * Construct a new ZooKeeper client, retrying indefinitely if it doesn't work right away. The + * caller is responsible for closing instances returned from this method. + * + * @param clientName a convenient name for logging its connection state changes + * @param connectString in the form of host1:port1,host2:port2/chroot/path + * @param timeout in milliseconds + * @param instanceSecret instance secret (may be null) + */ + public ZooSession(String clientName, String connectString, int timeout, String instanceSecret) { + // information needed to construct a ZooKeeper instance and add authentication + this.connectString = connectString; + this.timeout = timeout; + this.instanceSecret = instanceSecret; + + // information for logging which instance of ZooSession this is + this.sessionName = + String.format("%s[%s_%s]", getClass().getSimpleName(), clientName, UUID.randomUUID()); + this.connectCounter = new AtomicLong(); // incremented when we need to create a new delegate + this.zrw = new ZooReaderWriter(this); + } + + private ZooKeeper verifyConnected() { + if (closed.get()) { + throw new IllegalStateException(sessionName + " was closed"); + } + return delegate.updateAndGet(zk -> (zk != null && zk.getState().isAlive()) ? zk : reconnect()); + } + + private synchronized ZooKeeper reconnect() { + ZooKeeper zk; + if ((zk = delegate.get()) != null && zk.getState().isAlive()) { + return zk; + } + zk = null; + var reconnectName = String.format("%s#%s", sessionName, connectCounter.getAndIncrement()); + log.debug("{} (re-)connecting to {} with timeout {}{}", reconnectName, connectString, timeout, + instanceSecret == null ? "" : " with auth"); + final int TIME_BETWEEN_CONNECT_CHECKS_MS = 100; + int connectTimeWait = Math.min(10_000, timeout); + boolean tryAgain = true; + long sleepTime = 100; + + long startTime = System.nanoTime(); + + while (tryAgain) { + try { + zk = new ZooKeeper(connectString, timeout, new ZooSessionWatcher(reconnectName)); + // it may take some time to get connected to zookeeper if some of the servers are down + for (int i = 0; i < connectTimeWait / TIME_BETWEEN_CONNECT_CHECKS_MS && tryAgain; i++) { + if (zk.getState().isConnected()) { + if (instanceSecret != null) { + digestAuth(zk, instanceSecret); + } + tryAgain = false; + } else { + UtilWaitThread.sleep(TIME_BETWEEN_CONNECT_CHECKS_MS); + } + } + + } catch (IOException e) { + if (e instanceof UnknownHostException) { + /* + * Make sure we wait at least as long as the JVM TTL for negative DNS responses + */ + int ttl = AddressUtil.getAddressCacheNegativeTtl((UnknownHostException) e); + sleepTime = Math.max(sleepTime, (ttl + 1) * 1000L); + } + log.warn("Connection to zooKeeper failed, will try again in " + + String.format("%.2f secs", sleepTime / 1000.0), e); + } finally { + if (tryAgain && zk != null) { + closeZk(zk); + zk = null; + } + } + + long stopTime = System.nanoTime(); + long duration = NANOSECONDS.toMillis(stopTime - startTime); + + if (duration > 2L * timeout) { + throw new IllegalStateException("Failed to connect to zookeeper (" + connectString + + ") within 2x zookeeper timeout period " + timeout); + } + + if (tryAgain) { + if (2L * timeout < duration + sleepTime + connectTimeWait) { + sleepTime = 2L * timeout - duration - connectTimeWait; + } + if (sleepTime < 0) { + connectTimeWait -= sleepTime; + sleepTime = 0; + } + UtilWaitThread.sleep(sleepTime); + if (sleepTime < 10000) { + sleepTime = sleepTime + (long) (sleepTime * RANDOM.get().nextDouble()); + } + } + } + return zk; + } + + public void addAuthInfo(String scheme, byte[] auth) { + verifyConnected().addAuthInfo(scheme, auth); + } + + public String create(final String path, byte[] data, List acl, CreateMode createMode) + throws KeeperException, InterruptedException { + return verifyConnected().create(path, data, acl, createMode); + } + + public void delete(final String path, int version) throws InterruptedException, KeeperException { + verifyConnected().delete(path, version); + } + + public Stat exists(final String path, Watcher watcher) + throws KeeperException, InterruptedException { + return verifyConnected().exists(path, watcher); + } + + public List getACL(final String path, Stat stat) + throws KeeperException, InterruptedException { + return verifyConnected().getACL(path, stat); + } + + public List getChildren(final String path, Watcher watcher) + throws KeeperException, InterruptedException { + return verifyConnected().getChildren(path, watcher); + } + + public byte[] getData(final String path, Watcher watcher, Stat stat) + throws KeeperException, InterruptedException { + return verifyConnected().getData(path, watcher, stat); + } + + public long getSessionId() { + return verifyConnected().getSessionId(); + } + + public int getSessionTimeout() { + return verifyConnected().getSessionTimeout(); + } + + public void removeWatches(String path, Watcher watcher, WatcherType watcherType, boolean local) + throws InterruptedException, KeeperException { + verifyConnected().removeWatches(path, watcher, watcherType, local); + } + + public Stat setData(final String path, byte[] data, int version) + throws KeeperException, InterruptedException { + return verifyConnected().setData(path, data, version); + } + + public void sync(final String path, VoidCallback cb, Object ctx) { + verifyConnected().sync(path, cb, ctx); + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + closeZk(delegate.getAndSet(null)); + } + } + + public void addAccumuloDigestAuth(String auth) { + digestAuth(verifyConnected(), auth); + } + + public ZooReader asReader() { + return zrw; + } + + public ZooReaderWriter asReaderWriter() { + return zrw; + } + +} diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java index 991e4d2dba9..2450ec6ff66 100644 --- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java +++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java @@ -19,52 +19,27 @@ package org.apache.accumulo.core.clientImpl; import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; import static org.easymock.EasyMock.verify; -import java.util.UUID; - -import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooCache; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; public class ZookeeperLockCheckerTest { - private ClientContext context; - private ZooCache zc; - - @BeforeEach - public void setUp() { - var instanceId = InstanceId.of(UUID.randomUUID()); - zc = createMock(ZooCache.class); - context = createMock(ClientContext.class); - expect(context.getZooKeeperRoot()).andReturn(ZooUtil.getRoot(instanceId)).anyTimes(); - expect(context.getZooCache()).andReturn(zc).anyTimes(); - replay(context, zc); - } - - @AfterEach - public void tearDown() { - verify(context, zc); - } - @Test public void testInvalidateCache() { - var zklc = new ZookeeperLockChecker(context); + ZooCache zc = createMock(ZooCache.class); + String basePath = "/arbitrary/base/path"; - verify(zc); - reset(zc); - zc.clear(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/server"); + zc.clear(basePath + "/server"); expectLastCall().once(); replay(zc); + var zklc = new ZookeeperLockChecker(zc, basePath); zklc.invalidateCache("server"); + + verify(zc); } } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactoryTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactoryTest.java deleted file mode 100644 index 7db2dd9df2e..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheFactoryTest.java +++ /dev/null @@ -1,83 +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.fate.zookeeper; - -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNotSame; -import static org.junit.jupiter.api.Assertions.assertSame; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class ZooCacheFactoryTest { - private ZooCacheFactory zcf; - - @BeforeEach - public void setUp() { - zcf = new ZooCacheFactory(); - } - - @AfterEach - public void tearDown() { - zcf.reset(); - } - - @Test - public void testGetZooCache() { - String zks1 = "zk1"; - int timeout1 = 1000; - ZooCache zc1 = zcf.getZooCache(zks1, timeout1); - ZooCache zc1a = zcf.getZooCache(zks1, timeout1); - assertSame(zc1, zc1a); - - String zks2 = "zk2"; - int timeout2 = 1000; - ZooCache zc2 = zcf.getZooCache(zks2, timeout2); - assertNotSame(zc1, zc2); - - String zks3 = "zk1"; - int timeout3 = 2000; - ZooCache zc3 = zcf.getZooCache(zks3, timeout3); - assertNotSame(zc1, zc3); - } - - @Test - public void testGetNewZooCache() { - String zks1 = "zk1"; - int timeout1 = 1000; - ZooCache zc1 = zcf.getNewZooCache(zks1, timeout1); - assertNotNull(zc1); - ZooCache zc1a = zcf.getZooCache(zks1, timeout1); - assertNotSame(zc1, zc1a); - ZooCache zc1b = zcf.getNewZooCache(zks1, timeout1); - assertNotSame(zc1, zc1b); - assertNotSame(zc1a, zc1b); - } - - @Test - public void testReset() { - String zks1 = "zk1"; - int timeout1 = 1000; - ZooCache zc1 = zcf.getZooCache(zks1, timeout1); - zcf.reset(); - ZooCache zc1a = zcf.getZooCache(zks1, timeout1); - assertNotSame(zc1, zc1a); - } -} diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheTest.java index 39a21845682..22288aed3e9 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheTest.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooCacheTest.java @@ -20,11 +20,9 @@ import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.createStrictMock; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertArrayEquals; @@ -37,10 +35,11 @@ import java.util.List; import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat; +import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZooCacheWatcher; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.easymock.Capture; import org.easymock.EasyMock; @@ -52,19 +51,13 @@ public class ZooCacheTest { private static final byte[] DATA = {(byte) 1, (byte) 2, (byte) 3, (byte) 4}; private static final List CHILDREN = java.util.Arrays.asList("huey", "dewey", "louie"); - private ZooReader zr; - private ZooKeeper zk; + private ZooSession zk; private ZooCache zc; @BeforeEach public void setUp() { - zr = createMock(ZooReader.class); - zk = createStrictMock(ZooKeeper.class); - expect(zr.getZooKeeper()).andReturn(zk); - expectLastCall().anyTimes(); - replay(zr); - - zc = new ZooCache(zr, null); + zk = createStrictMock(ZooSession.class); + zc = new ZooCache(zk); } @Test @@ -220,7 +213,7 @@ public void testGetChildren_EatNoNode() throws Exception { verify(zk); } - private static class TestWatcher implements Watcher { + private static class TestWatcher implements ZooCacheWatcher { private final WatchedEvent expectedEvent; private boolean wasCalled; @@ -230,7 +223,7 @@ private static class TestWatcher implements Watcher { } @Override - public void process(WatchedEvent event) { + public void accept(WatchedEvent event) { assertSame(expectedEvent, event); wasCalled = true; } @@ -265,7 +258,7 @@ private void testWatchDataNode(byte[] initialData, Watcher.Event.EventType event WatchedEvent event = new WatchedEvent(eventType, Watcher.Event.KeeperState.SyncConnected, ZPATH); TestWatcher exw = new TestWatcher(event); - zc = new ZooCache(zr, exw); + zc = new ZooCache(zk, exw); Watcher w = watchData(initialData); w.process(event); @@ -303,7 +296,7 @@ public void testWatchDataNode_Expired() throws Exception { private void testWatchDataNode_Clear(Watcher.Event.KeeperState state) throws Exception { WatchedEvent event = new WatchedEvent(Watcher.Event.EventType.None, state, null); TestWatcher exw = new TestWatcher(event); - zc = new ZooCache(zr, exw); + zc = new ZooCache(zk, exw); Watcher w = watchData(DATA); assertTrue(zc.dataCached(ZPATH)); @@ -337,7 +330,7 @@ private void testWatchChildrenNode(List initialChildren, WatchedEvent event = new WatchedEvent(eventType, Watcher.Event.KeeperState.SyncConnected, ZPATH); TestWatcher exw = new TestWatcher(event); - zc = new ZooCache(zr, exw); + zc = new ZooCache(zk, exw); Watcher w = watchChildren(initialChildren); w.process(event); diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java index 0379c413fef..0dcabd6d10f 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java @@ -21,7 +21,6 @@ import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.createMockBuilder; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; @@ -32,6 +31,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter.Mutator; import org.apache.accumulo.core.util.Retry; import org.apache.accumulo.core.util.Retry.RetryFactory; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.BadVersionException; @@ -39,7 +39,6 @@ import org.apache.zookeeper.KeeperException.ConnectionLossException; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.KeeperException.SessionExpiredException; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -47,21 +46,23 @@ public class ZooReaderWriterTest { private ZooReaderWriter zrw; - private ZooKeeper zk; + private ZooSession zk; private RetryFactory retryFactory; private Retry retry; @BeforeEach public void setup() { - zk = createMock(ZooKeeper.class); - zrw = createMockBuilder(ZooReaderWriter.class) - .addMockedMethods("getRetryFactory", "getZooKeeper").createMock(); + zk = createMock(ZooSession.class); retryFactory = createMock(RetryFactory.class); retry = createMock(Retry.class); - - expect(zrw.getZooKeeper()).andReturn(zk).anyTimes(); - expect(zrw.getRetryFactory()).andReturn(retryFactory).anyTimes(); expect(retryFactory.createRetry()).andReturn(retry).anyTimes(); + zrw = new ZooReaderWriter(zk) { + @Override + protected RetryFactory getRetryFactory() { + return retryFactory; + } + }; + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); } @Test @@ -71,11 +72,11 @@ public void testDeleteSucceedOnInitialNoNode() throws Exception { zk.delete(path, -1); expectLastCall().andThrow(KeeperException.create(Code.NONODE)); - replay(zk, zrw, retryFactory, retry); + replay(zk, retryFactory, retry); zrw.delete(path); - verify(zk, zrw, retryFactory, retry); + verify(zk, retryFactory, retry); } @Test @@ -92,11 +93,11 @@ public void testDeleteSucceedOnRetry() throws Exception { zk.delete(path, -1); expectLastCall().andThrow(KeeperException.create(Code.NONODE)); - replay(zk, zrw, retryFactory, retry); + replay(zk, retryFactory, retry); zrw.delete(path); - verify(zk, zrw, retryFactory, retry); + verify(zk, retryFactory, retry); } @Test @@ -110,11 +111,11 @@ public void testMutateNodeCreationFails() throws Exception { expect(retry.canRetry()).andReturn(false); expect(retry.retriesCompleted()).andReturn(1L).once(); - replay(zk, zrw, retryFactory, retry); + replay(zk, retryFactory, retry); assertThrows(SessionExpiredException.class, () -> zrw.mutateOrCreate(path, value, mutator)); - verify(zk, zrw, retryFactory, retry); + verify(zk, retryFactory, retry); } @Test @@ -137,11 +138,11 @@ public void testMutateWithBadVersion() throws Exception { retry.waitForNextAttempt(anyObject(), anyObject()); expectLastCall().once(); - replay(zk, zrw, retryFactory, retry); + replay(zk, retryFactory, retry); assertArrayEquals(new byte[] {1}, zrw.mutateOrCreate(path, value, mutator)); - verify(zk, zrw, retryFactory, retry); + verify(zk, retryFactory, retry); } @Test @@ -167,10 +168,10 @@ public void testMutateWithRetryOnSetData() throws Exception { // Let 2nd setData succeed expect(zk.setData(path, mutatedBytes, 0)).andReturn(null); - replay(zk, zrw, retryFactory, retry); + replay(zk, retryFactory, retry); assertArrayEquals(new byte[] {1}, zrw.mutateOrCreate(path, value, mutator)); - verify(zk, zrw, retryFactory, retry); + verify(zk, retryFactory, retry); } } diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooSessionTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooSessionTest.java deleted file mode 100644 index efb73e95591..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooSessionTest.java +++ /dev/null @@ -1,43 +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.fate.zookeeper; - -import static java.util.concurrent.TimeUnit.SECONDS; -import static org.junit.jupiter.api.Assertions.assertThrows; - -import org.apache.zookeeper.ZooKeeper; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -public class ZooSessionTest { - - private static final int TIMEOUT_SECONDS = 10; - private static final String UNKNOWN_HOST = "hostname.that.should.not.exist.example.com:2181"; - - @Test - @Timeout(TIMEOUT_SECONDS * 4) - public void testUnknownHost() { - assertThrows(RuntimeException.class, () -> { - ZooKeeper session = ZooSession.connect(UNKNOWN_HOST, (int) SECONDS.toMillis(TIMEOUT_SECONDS), - null, null, null); - session.close(); - }); - } - -} diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java index 4f9315b9f5e..00dafa8e5cf 100644 --- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java +++ b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java @@ -18,12 +18,25 @@ */ package org.apache.accumulo.core.fate.zookeeper; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.isNull; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.data.InstanceId; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.data.ACL; @@ -66,4 +79,30 @@ private boolean validateACL(List acl) throws KeeperException.InvalidACLExce } return true; } + + @Test + public void fetchInstancesFromZk() throws Exception { + + String instAName = "INST_A"; + InstanceId instA = InstanceId.of(UUID.randomUUID()); + String instBName = "INST_B"; + InstanceId instB = InstanceId.of(UUID.randomUUID()); + + ZooSession zk = createMock(ZooSession.class); + String namePath = Constants.ZROOT + Constants.ZINSTANCES; + expect(zk.asReader()).andReturn(new ZooReader(zk)).anyTimes(); + expect(zk.getChildren(eq(namePath), isNull())).andReturn(List.of(instAName, instBName)).once(); + expect(zk.getData(eq(namePath + "/" + instAName), isNull(), isNull())) + .andReturn(instA.canonical().getBytes(UTF_8)).once(); + expect(zk.getData(eq(namePath + "/" + instBName), isNull(), isNull())) + .andReturn(instB.canonical().getBytes(UTF_8)).once(); + replay(zk); + + Map instanceMap = ZooUtil.getInstanceMap(zk); + + log.trace("id map returned: {}", instanceMap); + assertEquals(Map.of(instAName, instA, instBName, instB), instanceMap); + verify(zk); + } + } diff --git a/core/src/test/java/org/apache/accumulo/core/zookeeper/ZooSessionTest.java b/core/src/test/java/org/apache/accumulo/core/zookeeper/ZooSessionTest.java new file mode 100644 index 00000000000..2373961e83a --- /dev/null +++ b/core/src/test/java/org/apache/accumulo/core/zookeeper/ZooSessionTest.java @@ -0,0 +1,62 @@ +/* + * 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.zookeeper; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +class ZooSessionTest { + + private static final int ZK_TIMEOUT_SECONDS = 5; + + private String UNKNOWN_HOST = "hostname.that.should.not.exist.example.com:2181"; + private int millisTimeout = (int) SECONDS.toMillis(ZK_TIMEOUT_SECONDS); + + @Test + @Timeout(ZK_TIMEOUT_SECONDS * 4) + public void testConnectUnknownHost() { + + try (var zk = new ZooSession(getClass().getSimpleName(), UNKNOWN_HOST, millisTimeout, null)) { + var e = assertThrows(IllegalStateException.class, () -> { + zk.getSessionId(); + }); + assertTrue(e.getMessage().contains("Failed to connect to zookeeper (" + UNKNOWN_HOST + + ") within 2x zookeeper timeout period " + millisTimeout)); + } + } + + @Test + public void testClosed() { + ZooSession zk; + try (var zk2 = zk = new ZooSession("testClosed", UNKNOWN_HOST, millisTimeout, null)) { + assertNotNull(zk); + } + var e = assertThrows(IllegalStateException.class, () -> { + zk.getSessionId(); + }); + assertTrue(e.getMessage().startsWith("ZooSession[testClosed_"), e.getMessage()); + assertTrue(e.getMessage().endsWith("] was closed"), e.getMessage()); + } + +} diff --git a/minicluster/pom.xml b/minicluster/pom.xml index 9aaf67a8ef6..68e16e5c2a1 100644 --- a/minicluster/pom.xml +++ b/minicluster/pom.xml @@ -100,10 +100,6 @@ org.apache.zookeeper zookeeper - - org.apache.zookeeper - zookeeper-jute - org.slf4j slf4j-api diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java index 155b97f544d..7de3600a920 100644 --- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java +++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.List; import java.util.Properties; +import java.util.function.Supplier; import org.apache.accumulo.cluster.AccumuloCluster; import org.apache.accumulo.cluster.ClusterUser; @@ -44,6 +45,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import com.google.common.base.Suppliers; + import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; /** @@ -55,14 +58,14 @@ public class StandaloneAccumuloCluster implements AccumuloCluster { Collections.unmodifiableList(Arrays.asList(ServerType.MANAGER, ServerType.TABLET_SERVER, ServerType.GARBAGE_COLLECTOR, ServerType.MONITOR)); - private ClientInfo info; + private final ClientInfo info; private String accumuloHome, clientAccumuloConfDir, serverAccumuloConfDir, hadoopConfDir; - private Path tmp; - private List users; + private final Path tmp; + private final List users; private String clientCmdPrefix; private String serverCmdPrefix; - private SiteConfiguration siteConfig; - private ServerContext context; + private final SiteConfiguration siteConfig; + private final Supplier contextSupplier; @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "code runs in same security context as user who provided input file name") @@ -74,6 +77,7 @@ public StandaloneAccumuloCluster(ClientInfo info, Path tmp, List us this.serverAccumuloConfDir = serverAccumuloConfDir; siteConfig = SiteConfiguration.fromFile(new File(serverAccumuloConfDir, "accumulo.properties")).build(); + this.contextSupplier = Suppliers.memoize(() -> ServerContext.withClientInfo(siteConfig, info)); } public String getAccumuloHome() { @@ -129,12 +133,8 @@ public String getZooKeepers() { } @Override - public synchronized ServerContext getServerContext() { - if (context == null) { - context = ServerContext.override(siteConfig, info.getInstanceName(), info.getZooKeepers(), - info.getZooKeepersSessionTimeOut()); - } - return context; + public ServerContext getServerContext() { + return contextSupplier.get(); } @Override diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java index 2b1db848a16..7ea33e947a9 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java @@ -46,14 +46,12 @@ import java.util.Map.Entry; import java.util.Properties; import java.util.Set; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; @@ -81,6 +79,8 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.manager.state.SetGoalState; import org.apache.accumulo.minicluster.MiniAccumuloCluster; import org.apache.accumulo.minicluster.ServerType; @@ -101,10 +101,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.ZKUtil; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.ZooKeeper.States; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -487,7 +483,7 @@ public synchronized void start() throws IOException, InterruptedException { InstanceId instanceIdFromFile = VolumeManager.getInstanceIDFromHdfs(instanceIdPath, hadoopConf); - ZooReaderWriter zrw = getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getServerContext().getZooSession().asReaderWriter(); String rootPath = ZooUtil.getRoot(instanceIdFromFile); @@ -644,74 +640,48 @@ private void verifyUp() throws InterruptedException, IOException { waitForProcessStart(tsp, "TabletServer" + tsExpectedCount); } - try (ZooKeeper zk = new ZooKeeper(getZooKeepers(), 60000, event -> log.warn("{}", event))) { - - String secret = getSiteConfiguration().get(Property.INSTANCE_SECRET); - - while (!(zk.getState() == States.CONNECTED)) { - log.info("Waiting for ZK client to connect, state: {} - will retry", zk.getState()); - Thread.sleep(1000); - } - - String instanceId = null; + String secret = getSiteConfiguration().get(Property.INSTANCE_SECRET); + String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + getInstanceName(); + try (var zk = new ZooSession(MiniAccumuloClusterImpl.class.getSimpleName() + ".verifyUp()", + getZooKeepers(), 60000, secret)) { + var rdr = zk.asReader(); + InstanceId instanceId = null; for (int i = 0; i < numTries; i++) { - if (zk.getState() == States.CONNECTED) { - ZooUtil.digestAuth(zk, secret); - try { - final AtomicInteger rc = new AtomicInteger(); - final CountDownLatch waiter = new CountDownLatch(1); - zk.sync("/", (code, arg1, arg2) -> { - rc.set(code); - waiter.countDown(); - }, null); - waiter.await(); - Code code = Code.get(rc.get()); - if (code != Code.OK) { - throw KeeperException.create(code); - } - String instanceNamePath = - Constants.ZROOT + Constants.ZINSTANCES + "/" + config.getInstanceName(); - byte[] bytes = zk.getData(instanceNamePath, null, null); - instanceId = new String(bytes, UTF_8); - break; - } catch (KeeperException e) { - log.warn("Error trying to read instance id from zookeeper: " + e.getMessage()); - log.debug("Unable to read instance id from zookeeper.", e); - } - } else { - log.warn("ZK client not connected, state: {}", zk.getState()); + try { + // make sure it's up enough we can perform operations successfully + rdr.sync("/"); + // wait for the instance to be created + instanceId = InstanceId.of(new String(rdr.getData(instanceNamePath), UTF_8)); + break; + } catch (KeeperException e) { + log.warn("Error trying to read instance id from zookeeper: {}", e.getMessage()); + log.debug("Unable to read instance id from zookeeper.", e); } Thread.sleep(1000); } if (instanceId == null) { - for (int i = 0; i < numTries; i++) { - if (zk.getState() == States.CONNECTED) { - ZooUtil.digestAuth(zk, secret); - try { - log.warn("******* COULD NOT FIND INSTANCE ID - DUMPING ZK ************"); - log.warn("Connected to ZooKeeper: {}", getZooKeepers()); - log.warn("Looking for instanceId at {}", - Constants.ZROOT + Constants.ZINSTANCES + "/" + config.getInstanceName()); - ZKUtil.visitSubTreeDFS(zk, Constants.ZROOT, false, - (rc, path, ctx, name) -> log.warn("{}", path)); - log.warn("******* END ZK DUMP ************"); - } catch (KeeperException | InterruptedException e) { - log.error("Error dumping zk", e); - } - } - Thread.sleep(1000); + try { + log.warn("******* COULD NOT FIND INSTANCE ID - DUMPING ZK ************"); + log.warn("Connected to ZooKeeper: {}", getZooKeepers()); + log.warn("Looking for instanceId at {}", instanceNamePath); + ZKUtil.visitSubTreeDFS(zk, Constants.ZROOT, false, + (rc, path, ctx, name) -> log.warn("{}", path)); + log.warn("******* END ZK DUMP ************"); + } catch (KeeperException e) { + log.error("Error dumping zk", e); } throw new IllegalStateException("Unable to find instance id from zookeeper."); } - String rootPath = Constants.ZROOT + "/" + instanceId; + String rootPath = ZooUtil.getRoot(instanceId); int tsActualCount = 0; try { while (tsActualCount < tsExpectedCount) { tsActualCount = 0; - for (String child : zk.getChildren(rootPath + Constants.ZTSERVERS, null)) { - if (zk.getChildren(rootPath + Constants.ZTSERVERS + "/" + child, null).isEmpty()) { + String tserverPath = rootPath + Constants.ZTSERVERS; + for (String child : rdr.getChildren(tserverPath)) { + if (rdr.getChildren(tserverPath + "/" + child).isEmpty()) { log.info("TServer " + tsActualCount + " not yet present in ZooKeeper"); } else { tsActualCount++; @@ -725,7 +695,7 @@ private void verifyUp() throws InterruptedException, IOException { } try { - while (zk.getChildren(rootPath + Constants.ZMANAGER_LOCK, null).isEmpty()) { + while (rdr.getChildren(rootPath + Constants.ZMANAGER_LOCK).isEmpty()) { log.info("Manager not yet present in ZooKeeper"); Thread.sleep(500); } @@ -734,7 +704,7 @@ private void verifyUp() throws InterruptedException, IOException { } try { - while (zk.getChildren(rootPath + Constants.ZGC_LOCK, null).isEmpty()) { + while (rdr.getChildren(rootPath + Constants.ZGC_LOCK).isEmpty()) { log.info("GC not yet present in ZooKeeper"); Thread.sleep(500); } 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 a20b5c62bbd..b05519dfc4e 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 @@ -42,6 +42,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.clientImpl.ClientInfo; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; @@ -50,8 +51,6 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metrics.MetricsInfo; @@ -92,7 +91,6 @@ public class ServerContext extends ClientContext { private static final Logger log = LoggerFactory.getLogger(ServerContext.class); private final ServerInfo info; - private final ZooReaderWriter zooReaderWriter; private final ServerDirs serverDirs; private final Supplier propStore; private final Supplier zkUserPath; @@ -109,16 +107,18 @@ public class ServerContext extends ClientContext { private final Supplier metricsInfoSupplier; public ServerContext(SiteConfiguration siteConfig) { - this(new ServerInfo(siteConfig)); + this(ServerInfo.fromServerConfig(siteConfig)); } private ServerContext(ServerInfo info) { super(SingletonReservation.noop(), info, info.getSiteConfiguration(), Threads.UEH); this.info = info; - zooReaderWriter = new ZooReaderWriter(info.getSiteConfiguration()); serverDirs = info.getServerDirs(); - propStore = memoize(() -> ZooPropStore.initialize(getInstanceID(), getZooReaderWriter())); + // the PropStore shouldn't close the ZooKeeper, since ServerContext is responsible for that + @SuppressWarnings("resource") + var tmpPropStore = memoize(() -> ZooPropStore.initialize(getInstanceID(), getZooSession())); + propStore = tmpPropStore; zkUserPath = memoize(() -> ZooUtil.getRoot(getInstanceID()) + Constants.ZUSERS); tableManager = memoize(() -> new TableManager(this)); @@ -141,21 +141,24 @@ private ServerContext(ServerInfo info) { */ public static ServerContext initialize(SiteConfiguration siteConfig, String instanceName, InstanceId instanceID) { - return new ServerContext(new ServerInfo(siteConfig, instanceName, instanceID)); + return new ServerContext(ServerInfo.initialize(siteConfig, instanceName, instanceID)); + } + + /** + * Used by server-side utilities that have a client configuration. The instance name is obtained + * from the client configuration, and the instanceId is looked up in ZooKeeper from the name. + */ + public static ServerContext withClientInfo(SiteConfiguration siteConfig, ClientInfo info) { + return new ServerContext(ServerInfo.fromServerAndClientConfig(siteConfig, info)); } /** * Override properties for testing */ - public static ServerContext override(SiteConfiguration siteConfig, String instanceName, + public static ServerContext forTesting(SiteConfiguration siteConfig, String instanceName, String zooKeepers, int zkSessionTimeOut) { return new ServerContext( - new ServerInfo(siteConfig, instanceName, zooKeepers, zkSessionTimeOut)); - } - - @Override - public InstanceId getInstanceID() { - return info.getInstanceID(); + ServerInfo.forTesting(siteConfig, instanceName, zooKeepers, zkSessionTimeOut)); } public SiteConfiguration getSiteConfiguration() { @@ -211,15 +214,6 @@ public VolumeManager getVolumeManager() { return info.getVolumeManager(); } - @Override - public ZooReader getZooReader() { - return getZooReaderWriter(); - } - - public ZooReaderWriter getZooReaderWriter() { - return zooReaderWriter; - } - /** * Retrieve the SSL/TLS configuration for starting up a listening service */ @@ -317,7 +311,7 @@ public void waitForZookeeperAndHdfs() { log.info("Attempting to talk to zookeeper"); while (true) { try { - getZooReaderWriter().getChildren(Constants.ZROOT); + getZooSession().asReaderWriter().getChildren(Constants.ZROOT); break; } catch (InterruptedException | KeeperException ex) { log.info("Waiting for accumulo to be initialized"); 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 aa7d2149dde..2f8241d3df1 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 @@ -18,115 +18,135 @@ */ package org.apache.accumulo.server; -import static java.nio.charset.StandardCharsets.UTF_8; +import static com.google.common.base.Suppliers.memoize; +import static java.util.Objects.requireNonNull; import java.io.IOException; import java.io.UncheckedIOException; import java.util.Properties; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.function.ToIntFunction; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.clientImpl.ClientConfConverter; import org.apache.accumulo.core.clientImpl.ClientInfo; import org.apache.accumulo.core.clientImpl.Credentials; -import org.apache.accumulo.core.clientImpl.InstanceOperationsImpl; import org.apache.accumulo.core.conf.ClientProperty; 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.ZooCache; -import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory; 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; import org.apache.accumulo.server.security.SystemCredentials; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; public class ServerInfo implements ClientInfo { - private final SiteConfiguration siteConfig; - private final Configuration hadoopConf; - private final InstanceId instanceID; - private final String instanceName; - private final String zooKeepers; - private final int zooKeepersSessionTimeOut; - private final VolumeManager volumeManager; - private final ZooCache zooCache; - private final ServerDirs serverDirs; - private final Credentials credentials; - - ServerInfo(SiteConfiguration siteConfig, String instanceName, String zooKeepers, - int zooKeepersSessionTimeOut) { - SingletonManager.setMode(Mode.SERVER); - this.siteConfig = siteConfig; - this.hadoopConf = new Configuration(); - this.instanceName = instanceName; - this.zooKeepers = zooKeepers; - this.zooKeepersSessionTimeOut = zooKeepersSessionTimeOut; - try { - volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - zooCache = new ZooCacheFactory().getZooCache(zooKeepers, zooKeepersSessionTimeOut); - String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName; - byte[] iidb = zooCache.get(instanceNamePath); - if (iidb == null) { - throw new IllegalStateException( - "Instance name " + instanceName + " does not exist in zookeeper. " - + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list."); - } - instanceID = InstanceId.of(new String(iidb, UTF_8)); - if (zooCache.get(ZooUtil.getRoot(instanceID)) == null) { - if (instanceName == null) { - throw new IllegalStateException( - "Instance id " + instanceID + " does not exist in zookeeper"); + private static final Function GET_ZK_HOSTS_FROM_CONFIG = + si -> si.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST); + + private static final ToIntFunction GET_ZK_TIMEOUT_FROM_CONFIG = + si -> (int) si.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT); + + // set things up using the config file, the instanceId from HDFS, and ZK for the instanceName + static ServerInfo fromServerConfig(SiteConfiguration siteConfig) { + final Function instanceNameFromZk = si -> { + try (var zk = + si.getZooKeeperSupplier(ServerInfo.class.getSimpleName() + ".getInstanceId()").get()) { + return ZooUtil.getInstanceName(zk, si.getInstanceId()); } - throw new IllegalStateException("Instance id " + instanceID + " pointed to by the name " - + instanceName + " does not exist in zookeeper"); - } - serverDirs = new ServerDirs(siteConfig, hadoopConf); - credentials = SystemCredentials.get(instanceID, siteConfig); + }; + final Function instanceIdFromHdfs = si -> VolumeManager.getInstanceIDFromHdfs( + si.getServerDirs().getInstanceIdLocation(si.getVolumeManager().getFirst()), + si.getHadoopConf()); + return new ServerInfo(siteConfig, GET_ZK_HOSTS_FROM_CONFIG, GET_ZK_TIMEOUT_FROM_CONFIG, + instanceNameFromZk, instanceIdFromHdfs); + } + + // set things up using a provided instanceName and InstanceId to initialize the system, but still + // have a ServerContext that is functional without bootstrapping issues, so long as you don't call + // functions from it that require an instance to have already been initialized + static ServerInfo initialize(SiteConfiguration siteConfig, String instanceName, + InstanceId instanceId) { + requireNonNull(instanceName); + requireNonNull(instanceId); + return new ServerInfo(siteConfig, GET_ZK_HOSTS_FROM_CONFIG, GET_ZK_TIMEOUT_FROM_CONFIG, + si -> instanceName, si -> instanceId); + } + + // set things up using the config file, and the client config for a server-side CLI utility + static ServerInfo fromServerAndClientConfig(SiteConfiguration siteConfig, ClientInfo info) { + // ClientInfo.getInstanceId looks up the ID in ZK using the provided instance name + return new ServerInfo(siteConfig, si -> info.getZooKeepers(), + si -> info.getZooKeepersSessionTimeOut(), si -> info.getInstanceName(), + si -> info.getInstanceId()); + } + + static ServerInfo forTesting(SiteConfiguration siteConfig, String instanceName, String zooKeepers, + int zooKeepersSessionTimeOut) { + var props = new Properties(); + props.put(ClientProperty.INSTANCE_NAME, requireNonNull(instanceName)); + props.put(ClientProperty.INSTANCE_ZOOKEEPERS, requireNonNull(zooKeepers)); + props.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, zooKeepersSessionTimeOut); + return fromServerAndClientConfig(siteConfig, ClientInfo.from(props)); } - ServerInfo(SiteConfiguration config) { - SingletonManager.setMode(Mode.SERVER); - siteConfig = config; - hadoopConf = new Configuration(); - try { - volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - serverDirs = new ServerDirs(siteConfig, hadoopConf); - Path instanceIdPath = serverDirs.getInstanceIdLocation(volumeManager.getFirst()); - instanceID = VolumeManager.getInstanceIDFromHdfs(instanceIdPath, hadoopConf); - zooKeepers = config.get(Property.INSTANCE_ZK_HOST); - zooKeepersSessionTimeOut = (int) config.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT); - zooCache = new ZooCacheFactory().getZooCache(zooKeepers, zooKeepersSessionTimeOut); - instanceName = InstanceOperationsImpl.lookupInstanceName(zooCache, instanceID); - credentials = SystemCredentials.get(instanceID, siteConfig); - } - - ServerInfo(SiteConfiguration config, String instanceName, InstanceId instanceID) { + // required parameter + private final SiteConfiguration siteConfig; + + // suppliers for lazily loading + private final Supplier hadoopConf; + private final Supplier volumeManager; + private final Supplier serverDirs; + private final Supplier zooKeepers; + private final Supplier zooKeepersSessionTimeOut; // can't memoize IntSupplier + private final Supplier instanceId; + private final Supplier instanceName; + private final Supplier credentials; + private final Function zooSessionForName; + + // set up everything to be lazily loaded with memoized suppliers, so if nothing is used, the cost + // is low; to support different scenarios, plug in the functionality to retrieve certain items + // from ZooKeeper, HDFS, or from the input, using functions that take "this" and emit the desired + // object to be memoized on demand; these functions should not have cyclic dependencies on one + // another, but because things are lazily loaded, it is okay if one depends on another in one + // direction only + private ServerInfo(SiteConfiguration siteConfig, Function zkHostsFunction, + ToIntFunction zkTimeoutFunction, Function instanceNameFunction, + Function instanceIdFunction) { SingletonManager.setMode(Mode.SERVER); - siteConfig = config; - hadoopConf = new Configuration(); - try { - volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - this.instanceID = instanceID; - zooKeepers = config.get(Property.INSTANCE_ZK_HOST); - zooKeepersSessionTimeOut = (int) config.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT); - zooCache = new ZooCacheFactory().getZooCache(zooKeepers, zooKeepersSessionTimeOut); - this.instanceName = instanceName; - serverDirs = new ServerDirs(siteConfig, hadoopConf); - credentials = SystemCredentials.get(instanceID, siteConfig); + this.siteConfig = requireNonNull(siteConfig); + requireNonNull(zkHostsFunction); + requireNonNull(zkTimeoutFunction); + requireNonNull(instanceNameFunction); + requireNonNull(instanceIdFunction); + + this.hadoopConf = memoize(Configuration::new); + this.volumeManager = memoize(() -> { + try { + return VolumeManagerImpl.get(getSiteConfiguration(), getHadoopConf()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + this.serverDirs = memoize(() -> new ServerDirs(getSiteConfiguration(), getHadoopConf())); + this.credentials = + memoize(() -> SystemCredentials.get(getInstanceId(), getSiteConfiguration())); + + this.zooSessionForName = name -> new ZooSession(name, getZooKeepers(), + getZooKeepersSessionTimeOut(), getSiteConfiguration().get(Property.INSTANCE_SECRET)); + + // from here on, set up the suppliers based on what was passed in, to support different cases + this.zooKeepers = memoize(() -> zkHostsFunction.apply(this)); + this.zooKeepersSessionTimeOut = memoize(() -> zkTimeoutFunction.applyAsInt(this)); + this.instanceId = memoize(() -> instanceIdFunction.apply(this)); + this.instanceName = memoize(() -> instanceNameFunction.apply(this)); } public SiteConfiguration getSiteConfiguration() { @@ -134,21 +154,27 @@ public SiteConfiguration getSiteConfiguration() { } public VolumeManager getVolumeManager() { - return volumeManager; + return volumeManager.get(); + } + + @Override + public InstanceId getInstanceId() { + return instanceId.get(); } - public InstanceId getInstanceID() { - return instanceID; + @Override + public Supplier getZooKeeperSupplier(String clientName) { + return () -> zooSessionForName.apply(clientName); } @Override public String getZooKeepers() { - return zooKeepers; + return zooKeepers.get(); } @Override public int getZooKeepersSessionTimeOut() { - return zooKeepersSessionTimeOut; + return zooKeepersSessionTimeOut.get(); } @Override @@ -180,19 +206,20 @@ public Properties getProperties() { @Override public String getInstanceName() { - return instanceName; + return instanceName.get(); } public Credentials getCredentials() { - return credentials; + return credentials.get(); } @Override public Configuration getHadoopConf() { - return this.hadoopConf; + return hadoopConf.get(); } public ServerDirs getServerDirs() { - return serverDirs; + return serverDirs.get(); } + } diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java index 4eb7b4b184d..c83da8226b1 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java +++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java @@ -29,13 +29,9 @@ public class ServerUtilOpts extends ClientOpts { public synchronized ServerContext getServerContext() { if (context == null) { - if (getClientConfigFile() == null) { - context = new ServerContext(SiteConfiguration.auto()); - } else { - ClientInfo info = ClientInfo.from(getClientProps()); - context = ServerContext.override(SiteConfiguration.auto(), info.getInstanceName(), - info.getZooKeepers(), info.getZooKeepersSessionTimeOut()); - } + context = getClientConfigFile() == null ? new ServerContext(SiteConfiguration.auto()) + : ServerContext.withClientInfo(SiteConfiguration.auto(), + ClientInfo.from(getClientProps())); } return context; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java index c5842e5b663..2ae183122bf 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java @@ -24,6 +24,7 @@ import java.util.concurrent.Executor; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.PropStoreKey; @@ -44,9 +45,9 @@ public class ZooPropLoader implements CacheLoader,VersionedPrope // used to set watcher, does not react to events. private final PropStoreWatcher propStoreWatcher; - public ZooPropLoader(final ZooReaderWriter zrw, final VersionedPropCodec propCodec, + public ZooPropLoader(final ZooSession zk, final VersionedPropCodec propCodec, final PropStoreWatcher propStoreWatcher) { - this.zrw = zrw; + this.zrw = zk.asReaderWriter(); this.propCodec = propCodec; this.propStoreWatcher = propStoreWatcher; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java index 9f733e8495a..6369eda0afb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java @@ -31,6 +31,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.PropCache; @@ -56,38 +57,28 @@ public class ZooPropStore implements PropStore, PropChangeListener { private final PropCacheCaffeineImpl cache; private final ReadyMonitor zkReadyMon; - /** - * Create instance using ZooPropStore.Builder - * - * @param instanceId the instance id - * @param zrw a wrapper set of utilities for accessing ZooKeeper. - */ - private ZooPropStore(final InstanceId instanceId, final ZooReaderWriter zrw) { - this(instanceId, zrw, null, null, null); - } - /** * For testing create an instance with the optionally pass synthetic clock (Ticker), a * ReadyMonitor and a PropStore watcher allowing them to be mocked. If the optional components are * passed as null an internal instance is created. * * @param instanceId the instance id - * @param zrw a wrapper set of utilities for accessing ZooKeeper. + * @param zk a ZooKeeper client * @param monitor a ready monitor. Optional, if null, one is created. * @param watcher a watcher. Optional, if null, one is created. * @param ticker a synthetic clock used for testing. Optional, if null, one is created. */ - ZooPropStore(final InstanceId instanceId, final ZooReaderWriter zrw, final ReadyMonitor monitor, + ZooPropStore(final InstanceId instanceId, final ZooSession zk, final ReadyMonitor monitor, final PropStoreWatcher watcher, final Ticker ticker) { - this.zrw = zrw; + this.zrw = zk.asReaderWriter(); this.zkReadyMon = requireNonNullElseGet(monitor, - () -> new ReadyMonitor("prop-store", Math.round(zrw.getSessionTimeout() * 1.75))); + () -> new ReadyMonitor("prop-store", Math.round(zk.getSessionTimeout() * 1.75))); this.propStoreWatcher = requireNonNullElseGet(watcher, () -> new PropStoreWatcher(zkReadyMon)); - ZooPropLoader propLoader = new ZooPropLoader(zrw, codec, this.propStoreWatcher); + ZooPropLoader propLoader = new ZooPropLoader(zk, codec, this.propStoreWatcher); if (ticker == null) { this.cache = new PropCacheCaffeineImpl.Builder(propLoader).build(); @@ -115,8 +106,8 @@ private ZooPropStore(final InstanceId instanceId, final ZooReaderWriter zrw) { } public static ZooPropStore initialize(@NonNull final InstanceId instanceId, - @NonNull final ZooReaderWriter zrw) { - return new ZooPropStore(instanceId, zrw); + @NonNull final ZooSession zk) { + return new ZooPropStore(instanceId, zk, null, null, null); } @Override diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java index 5acc0bbc512..bd8bd9e0d5f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java @@ -21,7 +21,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.Constants.ZINSTANCES; import static org.apache.accumulo.core.Constants.ZROOT; -import static org.apache.accumulo.server.conf.util.ZooPropUtils.readInstancesFromZk; import static org.apache.accumulo.server.zookeeper.ZooAclUtil.checkWritableAuth; import static org.apache.accumulo.server.zookeeper.ZooAclUtil.extractAuthName; import static org.apache.accumulo.server.zookeeper.ZooAclUtil.translateZooPerm; @@ -50,8 +49,9 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.NamespacePropKey; @@ -63,8 +63,6 @@ import org.apache.accumulo.server.zookeeper.ZooAclUtil; import org.apache.accumulo.start.spi.KeywordExecutable; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZKUtil; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; @@ -141,7 +139,7 @@ void generateReport(final ServerContext context, final ZooInfoViewer.Opts opts) writer.println("Report Time: " + tsFormat.format(Instant.now())); writer.println("-----------------------------------------------"); if (opts.printInstanceIds) { - Map instanceMap = readInstancesFromZk(context.getZooReader()); + Map instanceMap = ZooUtil.getInstanceMap(context.getZooSession()); printInstanceIds(instanceMap, writer); } @@ -163,7 +161,7 @@ void generateReport(final ServerContext context, final ZooInfoViewer.Opts opts) private void printProps(final ServerContext context, final Opts opts, final PrintWriter writer) throws Exception { var iid = context.getInstanceID(); - var zooReader = context.getZooReader(); + var zooReader = context.getZooSession().asReader(); if (opts.printAllProps()) { log.info("all: {}", opts.printAllProps()); @@ -230,55 +228,57 @@ private void printAcls(final ServerContext context, final Opts opts, final Print writer.printf("ZooKeeper acls for instance ID: %s\n\n", iid.canonical()); - ZooKeeper zooKeeper = new ZooReaderWriter(opts.getSiteConfiguration()).getZooKeeper(); + var conf = opts.getSiteConfiguration(); + try (var zk = new ZooSession(getClass().getSimpleName(), conf)) { - String instanceRoot = ZooUtil.getRoot(iid); + String instanceRoot = ZooUtil.getRoot(iid); - final Stat stat = new Stat(); + final Stat stat = new Stat(); - recursiveAclRead(zooKeeper, ZROOT + ZINSTANCES, stat, aclMap); + recursiveAclRead(zk, ZROOT + ZINSTANCES, stat, aclMap); - recursiveAclRead(zooKeeper, instanceRoot, stat, aclMap); + recursiveAclRead(zk, instanceRoot, stat, aclMap); - // print formatting - aclMap.forEach((path, acl) -> { - if (acl == null) { - writer.printf("ERROR_ACCUMULO_MISSING_SOME: '%s' : none\n", path); - } else { - // sort for consistent presentation - acl.sort(Comparator.comparing(a -> a.getId().getId())); - ZooAclUtil.ZkAccumuloAclStatus aclStatus = checkWritableAuth(acl); - - String authStatus; - if (aclStatus.accumuloHasFull()) { - authStatus = "ACCUMULO_OKAY"; + // print formatting + aclMap.forEach((path, acl) -> { + if (acl == null) { + writer.printf("ERROR_ACCUMULO_MISSING_SOME: '%s' : none\n", path); } else { - authStatus = "ERROR_ACCUMULO_MISSING_SOME"; - } + // sort for consistent presentation + acl.sort(Comparator.comparing(a -> a.getId().getId())); + ZooAclUtil.ZkAccumuloAclStatus aclStatus = checkWritableAuth(acl); + + String authStatus; + if (aclStatus.accumuloHasFull()) { + authStatus = "ACCUMULO_OKAY"; + } else { + authStatus = "ERROR_ACCUMULO_MISSING_SOME"; + } - String otherUpdate; - if (aclStatus.othersMayUpdate() || aclStatus.anyCanRead()) { - otherUpdate = "NOT_PRIVATE"; - } else { - otherUpdate = "PRIVATE"; - } + String otherUpdate; + if (aclStatus.othersMayUpdate() || aclStatus.anyCanRead()) { + otherUpdate = "NOT_PRIVATE"; + } else { + otherUpdate = "PRIVATE"; + } - writer.printf("%s:%s %s", authStatus, otherUpdate, path); - boolean addSeparator = false; - for (ACL a : acl) { - if (addSeparator) { - writer.printf(","); + writer.printf("%s:%s %s", authStatus, otherUpdate, path); + boolean addSeparator = false; + for (ACL a : acl) { + if (addSeparator) { + writer.printf(","); + } + writer.printf(" %s:%s", translateZooPerm(a.getPerms()), extractAuthName(a)); + addSeparator = true; } - writer.printf(" %s:%s", translateZooPerm(a.getPerms()), extractAuthName(a)); - addSeparator = true; } - } - writer.println(""); - }); - writer.flush(); + writer.println(""); + }); + writer.flush(); + } } - private void recursiveAclRead(final ZooKeeper zooKeeper, final String rootPath, final Stat stat, + private void recursiveAclRead(final ZooSession zooKeeper, final String rootPath, final Stat stat, final Map> aclMap) { try { ZKUtil.visitSubTreeDFS(zooKeeper, rootPath, false, (rc, path, ctx, name) -> { @@ -347,7 +347,7 @@ private Map fetchNamespaceProps(InstanceId iid, ZooR private Map fetchTableProps(final ServerContext context, final List tables) { var iid = context.getInstanceID(); - var zooReader = context.getZooReader(); + var zooReader = context.getZooSession().asReader(); Set cmdOptTables = new TreeSet<>(tables); diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java index f86c8c6adab..e96a90370b2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java +++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java @@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.NamespacePropKey; @@ -83,10 +83,11 @@ public void execute(String[] args) throws Exception { ZooPropEditor.Opts opts = new ZooPropEditor.Opts(); opts.parseArgs(ZooPropEditor.class.getName(), args); - ZooReaderWriter zrw = new ZooReaderWriter(opts.getSiteConfiguration()); - var siteConfig = opts.getSiteConfiguration(); - try (ServerContext context = new ServerContext(siteConfig)) { + try (var zk = new ZooSession(getClass().getSimpleName(), siteConfig); + var context = new ServerContext(siteConfig)) { + var zrw = zk.asReaderWriter(); + PropStoreKey propKey = getPropKey(context, opts); switch (opts.getCmdMode()) { case SET: diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java deleted file mode 100644 index a892ca64a91..00000000000 --- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java +++ /dev/null @@ -1,84 +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.server.conf.util; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.Constants.ZINSTANCES; -import static org.apache.accumulo.core.Constants.ZROOT; - -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.UUID; - -import org.apache.accumulo.core.data.InstanceId; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ZooPropUtils { - private static final Logger LOG = LoggerFactory.getLogger(ZooPropUtils.class); - - private ZooPropUtils() {} - - /** - * Read the instance names and instance ids from ZooKeeper. The storage structure in ZooKeeper is: - * - *

-   *   /accumulo/instances/instance_name  - with the instance id stored as data.
-   * 
- * - * @return a map of (instance name, instance id) entries - */ - public static Map readInstancesFromZk(final ZooReader zooReader) { - String instanceRoot = ZROOT + ZINSTANCES; - Map idMap = new TreeMap<>(); - try { - List names = zooReader.getChildren(instanceRoot); - names.forEach(name -> { - InstanceId iid = getInstanceIdForName(zooReader, name); - idMap.put(name, iid); - }); - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("Interrupted reading instance name info from ZooKeeper", ex); - } catch (KeeperException ex) { - throw new IllegalStateException("Failed to read instance name info from ZooKeeper", ex); - } - return idMap; - } - - private static InstanceId getInstanceIdForName(ZooReader zooReader, String name) { - String instanceRoot = ZROOT + ZINSTANCES; - String path = ""; - try { - path = instanceRoot + "/" + name; - byte[] uuid = zooReader.getData(path); - return InstanceId.of(UUID.fromString(new String(uuid, UTF_8))); - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("Interrupted reading instance id from ZooKeeper", ex); - } catch (KeeperException ex) { - LOG.warn("Failed to read instance id for " + path); - return null; - } - } - -} 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 f849599233c..0b1a5045df3 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 @@ -47,6 +47,7 @@ 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; import org.apache.accumulo.server.AccumuloDataVersion; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.ServerDirs; @@ -525,7 +526,6 @@ public void execute(final String[] args) { Opts opts = new Opts(); opts.parseArgs("accumulo init", args); var siteConfig = SiteConfiguration.auto(); - ZooReaderWriter zoo = new ZooReaderWriter(siteConfig); SecurityUtil.serverLogin(siteConfig); Configuration hadoopConfig = new Configuration(); InitialConfiguration initConfig = new InitialConfiguration(hadoopConfig, siteConfig); @@ -539,7 +539,9 @@ public void execute(final String[] args) { success = addVolumes(fs, initConfig, serverDirs); } if (!opts.resetSecurity && !opts.addVolumes) { - success = doInit(zoo, opts, fs, initConfig); + try (var zk = new ZooSession(getClass().getSimpleName(), siteConfig)) { + success = doInit(zk.asReaderWriter(), opts, fs, initConfig); + } } } catch (IOException e) { log.error("Problem trying to get Volume configuration", e); diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java index 6be6e4feed3..7d2f769a044 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java @@ -95,7 +95,7 @@ void initialize(final ServerContext context, final boolean clearInstanceName, throws KeeperException, InterruptedException { // setup basic data in zookeeper - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); InstanceId instanceId = context.getInstanceID(); zoo.putPersistentData(Constants.ZROOT + Constants.ZINSTANCES, new byte[0], diff --git a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java index b2cc9336de0..da4f51dd356 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java +++ b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java @@ -93,7 +93,7 @@ public enum WalState { public WalStateManager(ServerContext context) { this.context = context; - this.zoo = context.getZooReaderWriter(); + this.zoo = context.getZooSession().asReaderWriter(); } private String root() throws WalMarkerException { diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java index 0e693ab1e1b..60adab9bc0f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java @@ -28,6 +28,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.thrift.ClientService; @@ -36,6 +37,7 @@ import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat; +import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZooCacheWatcher; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.manager.thrift.TabletServerStatus; @@ -57,13 +59,13 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NotEmptyException; import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Suppliers; import com.google.common.net.HostAndPort; -public class LiveTServerSet implements Watcher { +public class LiveTServerSet implements ZooCacheWatcher { public interface Listener { void update(LiveTServerSet current, Set deleted, Set added); @@ -73,7 +75,6 @@ public interface Listener { private final Listener cback; private final ServerContext context; - private ZooCache zooCache; public class TServerConnection { private final HostAndPort address; @@ -236,16 +237,16 @@ static class TServerInfo { // The set of entries in zookeeper without locks, and the first time each was noticed private final Map locklessServers = new HashMap<>(); + private final Supplier zcSupplier; + public LiveTServerSet(ServerContext context, Listener cback) { this.cback = cback; this.context = context; + this.zcSupplier = Suppliers.memoize(() -> new ZooCache(context.getZooSession(), this)); } - public synchronized ZooCache getZooCache() { - if (zooCache == null) { - zooCache = new ZooCache(context.getZooReader(), this); - } - return zooCache; + public ZooCache getZooCache() { + return zcSupplier.get(); } public synchronized void startListeningForTabletServerChanges() { @@ -280,7 +281,7 @@ public synchronized void scanServers() { private void deleteServerNode(String serverNode) throws InterruptedException, KeeperException { try { - context.getZooReaderWriter().delete(serverNode); + context.getZooSession().asReaderWriter().delete(serverNode); } catch (NotEmptyException ex) { // acceptable race condition: // tserver created the lock under this server's node after our last check @@ -334,7 +335,7 @@ private synchronized void checkServer(final Set updates, } @Override - public void process(WatchedEvent event) { + public void accept(WatchedEvent event) { // its important that these event are propagated by ZooCache, because this ensures when reading // zoocache that is has already processed the event and cleared @@ -432,7 +433,7 @@ public synchronized void remove(TServerInstance server) { log.info("Removing zookeeper lock for {}", server); String fullpath = context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + zPath; try { - context.getZooReaderWriter().recursiveDelete(fullpath, SKIP); + context.getZooSession().asReaderWriter().recursiveDelete(fullpath, SKIP); } catch (Exception e) { String msg = "error removing tablet server lock"; // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/DeadServerList.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/DeadServerList.java index 84a55f9f95c..5f99a0b9816 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/DeadServerList.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/DeadServerList.java @@ -43,9 +43,9 @@ public class DeadServerList { public DeadServerList(ServerContext context) { this.path = context.getZooKeeperRoot() + Constants.ZDEADTSERVERS; - zoo = context.getZooReaderWriter(); + zoo = context.getZooSession().asReaderWriter(); try { - context.getZooReaderWriter().mkdirs(path); + zoo.mkdirs(path); } catch (Exception ex) { log.error("Unable to make parent directories of " + path, ex); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java index 85c5992b4fc..ed1588707fc 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java @@ -94,7 +94,7 @@ public void mutate() { context.getZooCache().clear(zpath); // TODO examine implementation of getZooReaderWriter().mutate() - context.getZooReaderWriter().mutateOrCreate(zpath, new byte[0], currVal -> { + context.getZooSession().asReaderWriter().mutateOrCreate(zpath, new byte[0], currVal -> { String currJson = new String(currVal, UTF_8); var rtm = new RootTabletMetadata(currJson); rtm.update(mutation); diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java index c3697e726a1..c74ddfb49ee 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java @@ -96,7 +96,7 @@ public TabletsMutator mutateTablets() { private void mutateRootGcCandidates(Consumer mutator) { String zpath = context.getZooKeeperRoot() + ZROOT_TABLET_GC_CANDIDATES; try { - context.getZooReaderWriter().mutateOrCreate(zpath, new byte[0], currVal -> { + context.getZooSession().asReaderWriter().mutateOrCreate(zpath, new byte[0], currVal -> { String currJson = new String(currVal, UTF_8); RootGcCandidates rgcc = new RootGcCandidates(currJson); log.debug("Root GC candidates before change : {}", currJson); @@ -247,7 +247,7 @@ public void deleteGcCandidates(DataLevel level, Collection candidat @Override public Iterator getGcCandidates(DataLevel level) { if (level == DataLevel.ROOT) { - var zooReader = context.getZooReader(); + var zooReader = context.getZooSession().asReader(); byte[] jsonBytes; try { jsonBytes = diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java index 2f400255451..70d391aee55 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java @@ -41,7 +41,6 @@ import org.apache.accumulo.core.dataImpl.thrift.TColumn; import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent; import org.apache.accumulo.core.dataImpl.thrift.TRange; -import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.manager.thrift.FateOperation; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.security.Authorizations; @@ -74,7 +73,6 @@ public class SecurityOperation { private final PermissionHandler permHandle; private final boolean isKerberos; private final Supplier rootUserName; - private final ZooCache zooCache; private final String zkUserPath; protected final ServerContext context; @@ -105,8 +103,8 @@ protected SecurityOperation(ServerContext context, Authorizor author, Authentica PermissionHandler pm) { this.context = context; zkUserPath = context.zkUserPath(); - zooCache = new ZooCache(context.getZooReader(), null); - rootUserName = Suppliers.memoize(() -> new String(zooCache.get(zkUserPath), UTF_8)); + rootUserName = + Suppliers.memoize(() -> new String(context.getZooCache().get(zkUserPath), UTF_8)); authorizor = author; authenticator = authent; permHandle = pm; diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributor.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributor.java index ef4a1e204de..47ecefa9f88 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributor.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributor.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; @@ -47,15 +48,13 @@ public class ZooAuthenticationKeyDistributor { private static final Logger log = LoggerFactory.getLogger(ZooAuthenticationKeyDistributor.class); - private final ZooReaderWriter zk; + private final ZooReaderWriter zrw; private final String baseNode; private final AtomicBoolean initialized = new AtomicBoolean(false); - public ZooAuthenticationKeyDistributor(ZooReaderWriter zk, String baseNode) { - requireNonNull(zk); - requireNonNull(baseNode); - this.zk = zk; - this.baseNode = baseNode; + public ZooAuthenticationKeyDistributor(ZooSession zk, String baseNode) { + this.zrw = zk.asReaderWriter(); + this.baseNode = requireNonNull(baseNode); } /** @@ -67,8 +66,8 @@ public synchronized void initialize() throws KeeperException, InterruptedExcepti return; } - if (zk.exists(baseNode)) { - List acls = zk.getACL(baseNode); + if (zrw.exists(baseNode)) { + List acls = zrw.getACL(baseNode); if (acls.size() == 1) { ACL actualAcl = acls.get(0), expectedAcl = ZooUtil.PRIVATE.get(0); Id actualId = actualAcl.getId(); @@ -86,7 +85,7 @@ public synchronized void initialize() throws KeeperException, InterruptedExcepti throw new IllegalStateException( "Delegation token secret key node in ZooKeeper is not protected."); } else { - zk.putPrivatePersistentData(baseNode, new byte[0], NodeExistsPolicy.FAIL); + zrw.putPrivatePersistentData(baseNode, new byte[0], NodeExistsPolicy.FAIL); } initialized.set(true); @@ -100,7 +99,7 @@ public synchronized void initialize() throws KeeperException, InterruptedExcepti */ public List getCurrentKeys() throws KeeperException, InterruptedException { checkState(initialized.get(), "Not initialized"); - List children = zk.getChildren(baseNode); + List children = zrw.getChildren(baseNode); // Shortcircuit to avoid a list creation if (children.isEmpty()) { @@ -110,7 +109,7 @@ public List getCurrentKeys() throws KeeperException, Interrup // Deserialize each byte[] into an AuthenticationKey List keys = new ArrayList<>(children.size()); for (String child : children) { - byte[] data = zk.getData(qualifyPath(child)); + byte[] data = zrw.getData(qualifyPath(child)); if (data != null) { AuthenticationKey key = new AuthenticationKey(); try { @@ -138,7 +137,7 @@ public synchronized void advertise(AuthenticationKey newKey) // Make sure the node doesn't already exist String path = qualifyPath(newKey); - if (zk.exists(path)) { + if (zrw.exists(path)) { log.warn("AuthenticationKey with ID '{}' already exists in ZooKeeper", newKey.getKeyId()); return; } @@ -157,7 +156,7 @@ public synchronized void advertise(AuthenticationKey newKey) path); // Put it into ZK with the private ACL - zk.putPrivatePersistentData(path, serializedKey, NodeExistsPolicy.FAIL); + zrw.putPrivatePersistentData(path, serializedKey, NodeExistsPolicy.FAIL); } /** @@ -174,7 +173,7 @@ public synchronized void remove(AuthenticationKey key) requireNonNull(key); String path = qualifyPath(key); - if (!zk.exists(path)) { + if (!zrw.exists(path)) { log.warn("AuthenticationKey with ID '{}' doesn't exist in ZooKeeper", key.getKeyId()); return; } @@ -182,7 +181,7 @@ public synchronized void remove(AuthenticationKey key) log.debug("Removing AuthenticationKey with keyId {} from ZooKeeper at {}", key.getKeyId(), path); - zk.delete(path); + zrw.delete(path); } String qualifyPath(String keyId) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java index abc060da49a..8f765d06d12 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java @@ -23,6 +23,7 @@ import java.io.IOException; import org.apache.accumulo.core.fate.zookeeper.ZooReader; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.WatchedEvent; @@ -42,10 +43,10 @@ public class ZooAuthenticationKeyWatcher implements Watcher { private final ZooReader zk; private final String baseNode; - public ZooAuthenticationKeyWatcher(AuthenticationTokenSecretManager secretManager, ZooReader zk, + public ZooAuthenticationKeyWatcher(AuthenticationTokenSecretManager secretManager, ZooSession zk, String baseNode) { this.secretManager = secretManager; - this.zk = zk; + this.zk = zk.asReader(); this.baseNode = baseNode; } diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java index 7aa23061c54..f38cb30d611 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java @@ -59,7 +59,7 @@ public class KerberosAuthenticator implements Authenticator { @Override public void initialize(ServerContext context) { this.context = context; - zooCache = new ZooCache(context.getZooReader(), null); + zooCache = new ZooCache(context.getZooSession()); impersonation = new UserImpersonation(context.getConfiguration()); zkAuthenticator.initialize(context); zkUserPath = context.zkUserPath(); @@ -73,7 +73,7 @@ public boolean validSecurityHandlers() { private void createUserNodeInZk(String principal) throws KeeperException, InterruptedException { synchronized (zooCache) { zooCache.clear(); - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); zoo.putPrivatePersistentData(zkUserPath + "/" + principal, new byte[0], NodeExistsPolicy.FAIL); } @@ -83,7 +83,7 @@ private void createUserNodeInZk(String principal) throws KeeperException, Interr public void initializeSecurity(String principal, byte[] token) { try { // remove old settings from zookeeper first, if any - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); synchronized (zooCache) { zooCache.clear(); if (zoo.exists(zkUserPath)) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java index 10b7c4b880d..1bcb54f0e26 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java @@ -49,7 +49,7 @@ public final class ZKAuthenticator implements Authenticator { @Override public void initialize(ServerContext context) { this.context = context; - zooCache = new ZooCache(context.getZooReader(), null); + zooCache = new ZooCache(context.getZooSession()); zkUserPath = context.zkUserPath(); } @@ -57,7 +57,7 @@ public void initialize(ServerContext context) { public void initializeSecurity(String principal, byte[] token) { try { // remove old settings from zookeeper first, if any - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); synchronized (zooCache) { zooCache.clear(); if (zoo.exists(zkUserPath)) { @@ -84,7 +84,7 @@ private void constructUser(String user, byte[] pass) throws KeeperException, InterruptedException { synchronized (zooCache) { zooCache.clear(); - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); zoo.putPrivatePersistentData(zkUserPath + "/" + user, pass, NodeExistsPolicy.FAIL); } } @@ -122,7 +122,7 @@ public void dropUser(String user) throws AccumuloSecurityException { try { synchronized (zooCache) { zooCache.clear(); - context.getZooReaderWriter().recursiveDelete(zkUserPath + "/" + user, + context.getZooSession().asReaderWriter().recursiveDelete(zkUserPath + "/" + user, NodeMissingPolicy.FAIL); } } catch (InterruptedException e) { @@ -148,8 +148,9 @@ public void changePassword(String principal, AuthenticationToken token) try { synchronized (zooCache) { zooCache.clear(zkUserPath + "/" + principal); - context.getZooReaderWriter().putPrivatePersistentData(zkUserPath + "/" + principal, - ZKSecurityTool.createPass(pt.getPassword()), NodeExistsPolicy.OVERWRITE); + context.getZooSession().asReaderWriter().putPrivatePersistentData( + zkUserPath + "/" + principal, ZKSecurityTool.createPass(pt.getPassword()), + NodeExistsPolicy.OVERWRITE); } } catch (KeeperException e) { log.error("{}", e.getMessage(), e); diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java index 8c0691d86c0..5d18ac46253 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java @@ -49,7 +49,7 @@ public class ZKAuthorizor implements Authorizor { @Override public void initialize(ServerContext context) { this.context = context; - zooCache = new ZooCache(context.getZooReader(), null); + zooCache = new ZooCache(context.getZooSession()); zkUserPath = context.zkUserPath(); } @@ -70,7 +70,7 @@ public boolean validSecurityHandlers(Authenticator auth, PermissionHandler pm) { @Override public void initializeSecurity(TCredentials itw, String rootuser) throws AccumuloSecurityException { - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); // create the root user with no record-level authorizations try { @@ -90,7 +90,7 @@ public void initializeSecurity(TCredentials itw, String rootuser) @Override public void initUser(String user) throws AccumuloSecurityException { - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); try { zoo.putPersistentData(zkUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP); } catch (KeeperException e) { @@ -106,7 +106,7 @@ public void initUser(String user) throws AccumuloSecurityException { public void dropUser(String user) throws AccumuloSecurityException { try { synchronized (zooCache) { - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserAuths, NodeMissingPolicy.SKIP); zooCache.clear(zkUserPath + "/" + user); } @@ -129,7 +129,8 @@ public void changeAuthorizations(String user, Authorizations authorizations) try { synchronized (zooCache) { zooCache.clear(); - context.getZooReaderWriter().putPersistentData(zkUserPath + "/" + user + ZKUserAuths, + context.getZooSession().asReaderWriter().putPersistentData( + zkUserPath + "/" + user + ZKUserAuths, ZKSecurityTool.convertAuthorizations(authorizations), NodeExistsPolicy.OVERWRITE); } } catch (KeeperException e) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java index f5b3768a854..1cc933c347f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java @@ -64,8 +64,8 @@ public class ZKPermHandler implements PermissionHandler { @Override public void initialize(ServerContext context) { - zooCache = new ZooCache(context.getZooReader(), null); - zoo = context.getZooReaderWriter(); + zooCache = new ZooCache(context.getZooSession()); + zoo = context.getZooSession().asReaderWriter(); zkUserPath = context.zkUserPath(); ZKTablePath = context.getZooKeeperRoot() + Constants.ZTABLES; ZKNamespacePath = context.getZooKeeperRoot() + Constants.ZNAMESPACES; diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java index bd8f74e6f0c..7f987dc4a3f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java +++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java @@ -36,6 +36,7 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooCache; +import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZooCacheWatcher; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; @@ -50,7 +51,6 @@ import org.apache.accumulo.server.util.PropUtil; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.EventType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,7 +77,7 @@ public static void prepareNewNamespaceState(final ServerContext context, Namespa final PropStore propStore = context.getPropStore(); final InstanceId instanceId = context.getInstanceID(); log.debug("Creating ZooKeeper entries for new namespace {} (ID: {})", namespace, namespaceId); - context.getZooReaderWriter().putPersistentData( + context.getZooSession().asReaderWriter().putPersistentData( context.getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId, new byte[0], existsPolicy); var propKey = NamespacePropKey.of(instanceId, namespaceId); @@ -115,7 +115,7 @@ public static void prepareNewTableState(ZooReaderWriter zoo, PropStore propStore public static void prepareNewTableState(final ServerContext context, TableId tableId, NamespaceId namespaceId, String tableName, TableState state, NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException { - prepareNewTableState(context.getZooReaderWriter(), context.getPropStore(), + prepareNewTableState(context.getZooSession().asReaderWriter(), context.getPropStore(), context.getInstanceID(), tableId, namespaceId, tableName, state, existsPolicy); } @@ -123,8 +123,8 @@ public TableManager(ServerContext context) { this.context = context; zkRoot = context.getZooKeeperRoot(); instanceID = context.getInstanceID(); - zoo = context.getZooReaderWriter(); - zooStateCache = new ZooCache(zoo, new TableStateWatcher()); + zoo = context.getZooSession().asReaderWriter(); + zooStateCache = new ZooCache(context.getZooSession(), new TableStateWatcher()); updateTableStateCache(); } @@ -251,9 +251,9 @@ public boolean addObserver(TableObserver to) { } } - private class TableStateWatcher implements Watcher { + private class TableStateWatcher implements ZooCacheWatcher { @Override - public void process(WatchedEvent event) { + public void accept(WatchedEvent event) { if (log.isTraceEnabled()) { log.trace("{}", event); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java index 7aead5026ec..f88ee0fcaef 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java +++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java @@ -57,10 +57,11 @@ public synchronized String getNextName() { while (next >= maxAllocated) { final int allocate = getAllocation(); try { - byte[] max = context.getZooReaderWriter().mutateExisting(nextNamePath, currentValue -> { - long l = Long.parseLong(new String(currentValue, UTF_8), Character.MAX_RADIX); - return Long.toString(l + allocate, Character.MAX_RADIX).getBytes(UTF_8); - }); + byte[] max = + context.getZooSession().asReaderWriter().mutateExisting(nextNamePath, currentValue -> { + long l = Long.parseLong(new String(currentValue, UTF_8), Character.MAX_RADIX); + return Long.toString(l + allocate, Character.MAX_RADIX).getBytes(UTF_8); + }); maxAllocated = Long.parseLong(new String(max, UTF_8), Character.MAX_RADIX); next = maxAllocated - allocate; 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 4153e356fe2..b40aefc34a9 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 @@ -62,7 +62,6 @@ import org.apache.accumulo.core.fate.ReadOnlyTStore; import org.apache.accumulo.core.fate.ZooStore; import org.apache.accumulo.core.fate.zookeeper.ZooCache; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.manager.thrift.FateService; import org.apache.accumulo.core.metadata.AccumuloTable; @@ -866,7 +865,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zLockManagerPath = ServiceLock.path(zkRoot + Constants.ZMANAGER_LOCK); var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); String fateZkPath = zkRoot + Constants.ZFATE; - ZooReaderWriter zk = context.getZooReaderWriter(); + var zk = context.getZooSession(); ZooStore zs = new ZooStore<>(fateZkPath, zk); if (fateOpsCommand.cancel) { @@ -944,7 +943,7 @@ private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUti ReadOnlyTStore zs, ServiceLock.ServiceLockPath tableLocksPath) throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException { - ZooReaderWriter zk = context.getZooReaderWriter(); + var zk = context.getZooSession(); var transactions = admin.getStatus(zs, zk, tableLocksPath, null, null); // build id map - relies on unique ids for tables and namespaces diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java index 5a794c3eebe..c6523cf3ed5 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java @@ -28,13 +28,14 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.volume.Volume; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.ServerDirs; import org.apache.accumulo.server.fs.VolumeManager; @@ -100,56 +101,74 @@ private static void recurse(ZooReader zoo, String root, Visitor v) { private static void verifyAccumuloIsDown(ServerContext context, String oldPassword) throws Exception { - ZooReader zooReader = context.getZooReader().asWriter(oldPassword); - String root = context.getZooKeeperRoot(); - final List ephemerals = new ArrayList<>(); - recurse(zooReader, root, (zoo, path) -> { - Stat stat = zoo.getStatus(path); - if (stat.getEphemeralOwner() != 0) { - ephemerals.add(path); - } - }); - if (!ephemerals.isEmpty()) { - System.err.println("The following ephemeral nodes exist, something is still running:"); - for (String path : ephemerals) { - System.err.println(path); + var conf = context.getSiteConfiguration(); + try (var oldZk = + new ZooSession(ChangeSecret.class.getSimpleName() + ".verifyAccumuloIsDown(oldPassword)", + conf.get(Property.INSTANCE_ZK_HOST), + (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), oldPassword)) { + String root = context.getZooKeeperRoot(); + final List ephemerals = new ArrayList<>(); + recurse(oldZk.asReaderWriter(), root, (zoo, path) -> { + Stat stat = zoo.getStatus(path); + if (stat.getEphemeralOwner() != 0) { + ephemerals.add(path); + } + }); + if (!ephemerals.isEmpty()) { + System.err.println("The following ephemeral nodes exist, something is still running:"); + for (String path : ephemerals) { + System.err.println(path); + } + throw new Exception("Accumulo must be shut down in order to run this tool."); } - throw new Exception("Accumulo must be shut down in order to run this tool."); } } private static void rewriteZooKeeperInstance(final ServerContext context, final InstanceId newInstanceId, String oldPass, String newPass) throws Exception { - final ZooReaderWriter orig = context.getZooReader().asWriter(oldPass); - final ZooReaderWriter new_ = context.getZooReader().asWriter(newPass); - - String root = context.getZooKeeperRoot(); - recurse(orig, root, (zoo, path) -> { - String newPath = path.replace(context.getInstanceID().canonical(), newInstanceId.canonical()); - byte[] data = zoo.getData(path); - List acls = orig.getZooKeeper().getACL(path, new Stat()); - if (acls.containsAll(Ids.READ_ACL_UNSAFE)) { - new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL); - } else { - // upgrade - if (acls.containsAll(Ids.OPEN_ACL_UNSAFE)) { - // make user nodes private, they contain the user's password - String[] parts = path.split("/"); - if (parts[parts.length - 2].equals("users")) { - new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL); + var conf = context.getSiteConfiguration(); + try ( + var oldZk = new ZooSession( + ChangeSecret.class.getSimpleName() + ".rewriteZooKeeperInstance(oldPass)", + conf.get(Property.INSTANCE_ZK_HOST), + (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), oldPass); + var newZk = new ZooSession( + ChangeSecret.class.getSimpleName() + ".rewriteZooKeeperInstance(newPass)", + conf.get(Property.INSTANCE_ZK_HOST), + (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), newPass)) { + + final var orig = oldZk.asReaderWriter(); + final var new_ = newZk.asReaderWriter(); + String root = context.getZooKeeperRoot(); + recurse(orig, root, (zoo, path) -> { + String newPath = + path.replace(context.getInstanceID().canonical(), newInstanceId.canonical()); + byte[] data = zoo.getData(path); + List acls = oldZk.getACL(path, new Stat()); + if (acls.containsAll(Ids.READ_ACL_UNSAFE)) { + new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL); + } else { + // upgrade + if (acls.containsAll(Ids.OPEN_ACL_UNSAFE)) { + // make user nodes private, they contain the user's password + String[] parts = path.split("/"); + if (parts[parts.length - 2].equals("users")) { + new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL); + } else { + // everything else can have the readable acl + new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL); + } } else { - // everything else can have the readable acl - new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL); + new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL); } - } else { - new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL); } - } - }); - String path = Constants.ZROOT + Constants.ZINSTANCES + "/" + context.getInstanceName(); - orig.recursiveDelete(path, NodeMissingPolicy.SKIP); - new_.putPersistentData(path, newInstanceId.canonical().getBytes(UTF_8), - NodeExistsPolicy.OVERWRITE); + }); + String path = Constants.ZROOT + Constants.ZINSTANCES + "/" + context.getInstanceName(); + orig.recursiveDelete(path, NodeMissingPolicy.SKIP); + new_.putPersistentData(path, newInstanceId.canonical().getBytes(UTF_8), + NodeExistsPolicy.OVERWRITE); + + } } private static void updateHdfs(ServerDirs serverDirs, VolumeManager fs, InstanceId newInstanceId) @@ -201,7 +220,13 @@ private static void checkHdfsAccessPermissions(FileStatus stat, FsAction mode) t } private static void deleteInstance(ServerContext context, String oldPass) throws Exception { - ZooReaderWriter orig = context.getZooReader().asWriter(oldPass); - orig.recursiveDelete(context.getZooKeeperRoot(), NodeMissingPolicy.SKIP); + var conf = context.getSiteConfiguration(); + try (var oldZk = new ZooSession(ChangeSecret.class.getSimpleName() + ".deleteInstance()", + conf.get(Property.INSTANCE_ZK_HOST), + (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), oldPass)) { + + var orig = oldZk.asReaderWriter(); + orig.recursiveDelete(context.getZooKeeperRoot(), NodeMissingPolicy.SKIP); + } } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java index 0fdbe4972a8..2b94c45496a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java @@ -27,7 +27,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; @@ -41,9 +40,8 @@ public class DeleteZooInstance { public static void execute(final ServerContext context, final boolean clean, final String instance, final String auth) throws InterruptedException, KeeperException { - final ZooReaderWriter zk = context.getZooReaderWriter(); if (auth != null) { - ZooUtil.digestAuth(zk.getZooKeeper(), auth); + context.getZooSession().addAccumuloDigestAuth(auth); } if (clean) { @@ -52,28 +50,29 @@ public static void execute(final ServerContext context, final boolean clean, throw new IllegalArgumentException( "Cannot set clean flag to true and also an instance name"); } - cleanAllOld(context, zk); + cleanAllOld(context); } else { // If all old is false then we require a specific instance Objects.requireNonNull(instance, "Instance name must not be null"); - removeInstance(context, zk, instance); + removeInstance(context, instance); } } - private static void removeInstance(ServerContext context, final ZooReaderWriter zk, - final String instance) throws InterruptedException, KeeperException { + private static void removeInstance(ServerContext context, final String instance) + throws InterruptedException, KeeperException { + var zrw = context.getZooSession().asReaderWriter(); // try instance name: - Set instances = new HashSet<>(getInstances(zk)); - Set uuids = new HashSet<>(zk.getChildren(Constants.ZROOT)); + Set instances = new HashSet<>(getInstances(zrw)); + Set uuids = new HashSet<>(zrw.getChildren(Constants.ZROOT)); uuids.remove("instances"); if (instances.contains(instance)) { String path = getInstancePath(instance); - byte[] data = zk.getData(path); + byte[] data = zrw.getData(path); if (data != null) { final String instanceId = new String(data, UTF_8); if (checkCurrentInstance(context, instance, instanceId)) { - deleteRetry(zk, path); - deleteRetry(zk, getRootChildPath(instanceId)); + deleteRetry(zrw, path); + deleteRetry(zrw, getRootChildPath(instanceId)); System.out.println("Deleted instance: " + instance); } } @@ -81,33 +80,34 @@ private static void removeInstance(ServerContext context, final ZooReaderWriter // look for the real instance name for (String zkInstance : instances) { String path = getInstancePath(zkInstance); - byte[] data = zk.getData(path); + byte[] data = zrw.getData(path); if (data != null) { final String instanceId = new String(data, UTF_8); if (instance.equals(instanceId) && checkCurrentInstance(context, instance, instanceId)) { - deleteRetry(zk, path); + deleteRetry(zrw, path); System.out.println("Deleted instance: " + instance); } } } - deleteRetry(zk, getRootChildPath(instance)); + deleteRetry(zrw, getRootChildPath(instance)); } } - private static void cleanAllOld(ServerContext context, final ZooReaderWriter zk) + private static void cleanAllOld(ServerContext context) throws InterruptedException, KeeperException { - for (String child : zk.getChildren(Constants.ZROOT)) { + var zrw = context.getZooSession().asReaderWriter(); + for (String child : zrw.getChildren(Constants.ZROOT)) { if (Constants.ZINSTANCES.equals("/" + child)) { - for (String instanceName : getInstances(zk)) { + for (String instanceName : getInstances(zrw)) { String instanceNamePath = getInstancePath(instanceName); - byte[] id = zk.getData(instanceNamePath); + byte[] id = zrw.getData(instanceNamePath); if (id != null && !new String(id, UTF_8).equals(context.getInstanceID().canonical())) { - deleteRetry(zk, instanceNamePath); + deleteRetry(zrw, instanceNamePath); System.out.println("Deleted instance: " + instanceName); } } } else if (!child.equals(context.getInstanceID().canonical())) { - deleteRetry(zk, getRootChildPath(child)); + deleteRetry(zrw, getRootChildPath(child)); } } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java index f3b18eda536..bf9e80574e1 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java @@ -25,6 +25,7 @@ import org.apache.accumulo.core.cli.ConfigOpts; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.start.spi.KeywordExecutable; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -35,7 +36,7 @@ @AutoService(KeywordExecutable.class) public class DumpZookeeper implements KeywordExecutable { - private static ZooReaderWriter zk = null; + private static ZooReaderWriter zrw = null; @Override public String keyword() { @@ -72,11 +73,14 @@ public void execute(String[] args) throws KeeperException, InterruptedException opts.parseArgs(DumpZookeeper.class.getName(), args); PrintStream out = System.out; - zk = new ZooReaderWriter(opts.getSiteConfiguration()); - if (opts.xml) { - writeXml(out, opts.root); - } else { - writeHumanReadable(out, opts.root); + var conf = opts.getSiteConfiguration(); + try (var zk = new ZooSession(getClass().getSimpleName(), conf)) { + zrw = zk.asReaderWriter(); + if (opts.xml) { + writeXml(out, opts.root); + } else { + writeHumanReadable(out, opts.root); + } } } @@ -87,7 +91,7 @@ public static void main(String[] args) throws KeeperException, InterruptedExcept private static void writeXml(PrintStream out, String root) throws KeeperException, InterruptedException { write(out, 0, "", root); - for (String child : zk.getChildren(root)) { + for (String child : zrw.getChildren(root)) { if (!child.equals("zookeeper")) { childXml(out, root, child, 1); } @@ -101,7 +105,7 @@ private static void childXml(PrintStream out, String root, String child, int ind if (root.endsWith("/")) { path = root + child; } - Stat stat = zk.getStatus(path); + Stat stat = zrw.getStatus(path); if (stat == null) { return; } @@ -125,7 +129,7 @@ private static void childXml(PrintStream out, String root, String child, int ind write(out, indent, "<%s name='%s' encoding='%s' value='%s'>", type, child, value.encoding, value.value); } - for (String c : zk.getChildren(path)) { + for (String c : zrw.getChildren(path)) { childXml(out, path, c, indent + 1); } write(out, indent, ""); @@ -133,7 +137,7 @@ private static void childXml(PrintStream out, String root, String child, int ind } private static Encoded value(String path) throws KeeperException, InterruptedException { - byte[] data = zk.getData(path); + byte[] data = zrw.getData(path); for (byte element : data) { // does this look like simple ascii? if (element < ' ' || element > '~') { @@ -153,7 +157,7 @@ private static void write(PrintStream out, int indent, String fmt, Object... arg private static void writeHumanReadable(PrintStream out, String root) throws KeeperException, InterruptedException { write(out, 0, "%s:", root); - for (String child : zk.getChildren(root)) { + for (String child : zrw.getChildren(root)) { if (!child.equals("zookeeper")) { childHumanReadable(out, root, child, 1); } @@ -166,7 +170,7 @@ private static void childHumanReadable(PrintStream out, String root, String chil if (root.endsWith("/")) { path = root + child; } - Stat stat = zk.getStatus(path); + Stat stat = zrw.getStatus(path); if (stat == null) { return; } @@ -180,7 +184,7 @@ private static void childHumanReadable(PrintStream out, String root, String chil write(out, indent, "%s: %s", node, value(path).value); } if (stat.getNumChildren() > 0) { - for (String c : zk.getChildren(path)) { + for (String c : zrw.getChildren(path)) { childHumanReadable(out, path, c, indent + 1); } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java index 24e7b218b73..780c3babd34 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java @@ -39,6 +39,7 @@ import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,7 +68,7 @@ static class Opts extends Help { static Opts opts = new Opts(); static int errors = 0; - public static void main(String[] args) { + public static void main(String[] args) throws InterruptedException { opts.parseArgs(ListInstances.class.getName(), args); if (opts.keepers == null) { @@ -80,43 +81,46 @@ public static void main(String[] args) { boolean printErrors = opts.printErrors; listInstances(keepers, printAll, printErrors); - } - static synchronized void listInstances(String keepers, boolean printAll, boolean printErrors) { + static synchronized void listInstances(String keepers, boolean printAll, boolean printErrors) + throws InterruptedException { errors = 0; System.out.println("INFO : Using ZooKeepers " + keepers); - ZooReader rdr = new ZooReader(keepers, ZOOKEEPER_TIMER_MILLIS); - ZooCache cache = new ZooCache(rdr, null); + try (var zk = new ZooSession(ListInstances.class.getSimpleName(), keepers, + ZOOKEEPER_TIMER_MILLIS, null)) { + ZooReader rdr = zk.asReader(); + ZooCache cache = new ZooCache(zk); - TreeMap instanceNames = getInstanceNames(rdr, printErrors); + TreeMap instanceNames = getInstanceNames(rdr, printErrors); - System.out.println(); - printHeader(); + System.out.println(); + printHeader(); - for (Entry entry : instanceNames.entrySet()) { - printInstanceInfo(cache, entry.getKey(), entry.getValue(), printErrors); - } + for (Entry entry : instanceNames.entrySet()) { + printInstanceInfo(cache, entry.getKey(), entry.getValue(), printErrors); + } - TreeSet instancedIds = getInstanceIDs(rdr, printErrors); - instancedIds.removeAll(instanceNames.values()); + TreeSet instancedIds = getInstanceIDs(rdr, printErrors); + instancedIds.removeAll(instanceNames.values()); - if (printAll) { - for (InstanceId uuid : instancedIds) { - printInstanceInfo(cache, null, uuid, printErrors); + if (printAll) { + for (InstanceId uuid : instancedIds) { + printInstanceInfo(cache, null, uuid, printErrors); + } + } else if (!instancedIds.isEmpty()) { + System.out.println(); + System.out.println("INFO : " + instancedIds.size() + + " unnamed instances were not printed, run with --print-all to see all instances"); + } else { + System.out.println(); } - } else if (!instancedIds.isEmpty()) { - System.out.println(); - System.out.println("INFO : " + instancedIds.size() - + " unnamed instances were not printed, run with --print-all to see all instances"); - } else { - System.out.println(); - } - if (!printErrors && errors > 0) { - System.err.println( - "WARN : There were " + errors + " errors, run with --print-errors to see more info"); + if (!printErrors && errors > 0) { + System.err.println( + "WARN : There were " + errors + " errors, run with --print-errors to see more info"); + } } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java index ff09cd3b45f..ab33f0c7dcc 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java @@ -32,6 +32,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.xml.sax.Attributes; import org.xml.sax.helpers.DefaultHandler; @@ -109,20 +110,21 @@ private void create(String path, String value, String encoding) { justification = "code runs in same security context as user who provided input") public static void execute(final AccumuloConfiguration conf, final String file, final boolean overwrite) throws Exception { - var zoo = new ZooReaderWriter(conf); + try (var zk = new ZooSession(RestoreZookeeper.class.getSimpleName(), conf)) { + var zrw = zk.asReaderWriter(); - InputStream in = System.in; - if (file != null) { - in = new FileInputStream(file); - } + InputStream in = System.in; + if (file != null) { + in = new FileInputStream(file); + } - SAXParserFactory factory = SAXParserFactory.newInstance(); - // Prevent external entities by failing on any doctypes. We don't expect any doctypes, so this - // is a simple switch to remove any chance of external entities causing problems. - factory.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true); - SAXParser parser = factory.newSAXParser(); - parser.parse(in, new Restore(zoo, overwrite)); - in.close(); + SAXParserFactory factory = SAXParserFactory.newInstance(); + // Prevent external entities by failing on any doctypes. We don't expect any doctypes, so this + // is a simple switch to remove any chance of external entities causing problems. + factory.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true); + SAXParser parser = factory.newSAXParser(); + parser.parse(in, new Restore(zrw, overwrite)); + in.close(); + } } - } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java index 07bc2dbca2d..93e86ec2c19 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java @@ -59,7 +59,7 @@ public ServiceStatusCmd() {} */ public void execute(final ServerContext context, final Opts opts) { - ZooReader zooReader = context.getZooReader(); + ZooReader zooReader = context.getZooSession().asReader(); final String zooRoot = context.getZooKeeperRoot(); LOG.trace("zooRoot: {}", zooRoot); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java index 8d36913c5f3..41ceb09a03c 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java @@ -36,7 +36,7 @@ public static void execute(final ServerContext context, final String lock, final String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS; ZooCache cache = context.getZooCache(); - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); if (delete == null) { List tabletServers = zoo.getChildren(tserverPath); 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 87c1eacf0ee..ed4a403a3a2 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 @@ -32,6 +32,7 @@ import org.apache.accumulo.core.singletons.SingletonManager; import org.apache.accumulo.core.singletons.SingletonManager.Mode; import org.apache.accumulo.core.volume.VolumeConfiguration; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.security.SecurityUtil; import org.apache.accumulo.start.spi.KeywordExecutable; @@ -95,8 +96,8 @@ public void execute(String[] args) throws Exception { return; } - try { - var siteConf = SiteConfiguration.auto(); + var siteConf = SiteConfiguration.auto(); + try (var zk = new ZooSession(getClass().getSimpleName(), siteConf)) { // Login as the server on secure HDFS if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) { SecurityUtil.serverLogin(siteConf); @@ -105,13 +106,13 @@ public void execute(String[] args) throws Exception { String volDir = VolumeConfiguration.getVolumeUris(siteConf).iterator().next(); Path instanceDir = new Path(volDir, "instance_id"); InstanceId iid = VolumeManager.getInstanceIDFromHdfs(instanceDir, new Configuration()); - ZooReaderWriter zoo = new ZooReaderWriter(siteConf); + var zrw = zk.asReaderWriter(); if (opts.zapManager) { String managerLockPath = ZooUtil.getRoot(iid) + Constants.ZMANAGER_LOCK; try { - zapDirectory(zoo, managerLockPath, opts); + zapDirectory(zrw, managerLockPath, opts); } catch (KeeperException | InterruptedException e) { e.printStackTrace(); } @@ -120,16 +121,16 @@ public void execute(String[] args) throws Exception { if (opts.zapTservers) { String tserversPath = ZooUtil.getRoot(iid) + Constants.ZTSERVERS; try { - List children = zoo.getChildren(tserversPath); + List children = zrw.getChildren(tserversPath); for (String child : children) { message("Deleting " + tserversPath + "/" + child + " from zookeeper", opts); if (opts.zapManager) { - zoo.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP); } else { var zLockPath = ServiceLock.path(tserversPath + "/" + child); - if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { - if (!ServiceLock.deleteLock(zoo, zLockPath, "tserver")) { + if (!zrw.getChildren(zLockPath.toString()).isEmpty()) { + if (!ServiceLock.deleteLock(zrw, zLockPath, "tserver")) { message("Did not delete " + tserversPath + "/" + child, opts); } } @@ -143,8 +144,8 @@ public void execute(String[] args) throws Exception { if (opts.zapCoordinators) { final String coordinatorPath = ZooUtil.getRoot(iid) + Constants.ZCOORDINATOR_LOCK; try { - if (zoo.exists(coordinatorPath)) { - zapDirectory(zoo, coordinatorPath, opts); + if (zrw.exists(coordinatorPath)) { + zapDirectory(zrw, coordinatorPath, opts); } } catch (KeeperException | InterruptedException e) { log.error("Error deleting coordinator from zookeeper, {}", e.getMessage(), e); @@ -154,11 +155,11 @@ public void execute(String[] args) throws Exception { if (opts.zapCompactors) { String compactorsBasepath = ZooUtil.getRoot(iid) + Constants.ZCOMPACTORS; try { - if (zoo.exists(compactorsBasepath)) { - List queues = zoo.getChildren(compactorsBasepath); + if (zrw.exists(compactorsBasepath)) { + List queues = zrw.getChildren(compactorsBasepath); for (String queue : queues) { message("Deleting " + compactorsBasepath + "/" + queue + " from zookeeper", opts); - zoo.recursiveDelete(compactorsBasepath + "/" + queue, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(compactorsBasepath + "/" + queue, NodeMissingPolicy.SKIP); } } } catch (KeeperException | InterruptedException e) { @@ -170,14 +171,14 @@ public void execute(String[] args) throws Exception { if (opts.zapScanServers) { String sserversPath = ZooUtil.getRoot(iid) + Constants.ZSSERVERS; try { - if (zoo.exists(sserversPath)) { - List children = zoo.getChildren(sserversPath); + if (zrw.exists(sserversPath)) { + List children = zrw.getChildren(sserversPath); for (String child : children) { message("Deleting " + sserversPath + "/" + child + " from zookeeper", opts); var zLockPath = ServiceLock.path(sserversPath + "/" + child); - if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { - ServiceLock.deleteLock(zoo, zLockPath); + if (!zrw.getChildren(zLockPath.toString()).isEmpty()) { + ServiceLock.deleteLock(zrw, zLockPath); } } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java index 3b4cd1112b1..b6575b9e86f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/RootMetadataCheckRunner.java @@ -87,7 +87,7 @@ public Admin.CheckCommand.CheckStatus runCheck(ServerContext context, ServerUtil log.trace("********** Looking for invalid columns **********"); final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; - final String json = new String(context.getZooReader().getData(path), UTF_8); + final String json = new String(context.getZooSession().asReader().getData(path), UTF_8); final var rtm = new RootTabletMetadata(json); status = checkColumns(context, rtm.toKeyValues().iterator(), status); @@ -100,7 +100,7 @@ public Admin.CheckCommand.CheckStatus checkRequiredColumns(ServerContext context Admin.CheckCommand.CheckStatus status) throws TableNotFoundException, InterruptedException, KeeperException { final String path = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET; - final String json = new String(context.getZooReader().getData(path), UTF_8); + final String json = new String(context.getZooSession().asReader().getData(path), UTF_8); final var rtm = new RootTabletMetadata(json); final Set rowsSeen = new HashSet<>(); final Set requiredColFQs = new HashSet<>(requiredColFQs()); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java index 9161c9867da..9ab3bee2ab2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/checkCommand/TableLocksCheckRunner.java @@ -26,7 +26,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.ZooStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; @@ -61,14 +60,15 @@ private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext cont final String zkRoot = context.getZooKeeperRoot(); final var zTableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS); final String fateZkPath = zkRoot + Constants.ZFATE; - final ZooReaderWriter zk = context.getZooReaderWriter(); + final var zk = context.getZooSession(); final ZooStore zs = new ZooStore<>(fateZkPath, zk); log.trace("Ensuring table and namespace locks are valid..."); var tableIds = context.tableOperations().tableIdMap().values(); var namespaceIds = context.namespaceOperations().namespaceIdMap().values(); - List lockedIds = zk.getChildren(zTableLocksPath.toString()); + List lockedIds = + context.getZooSession().asReader().getChildren(zTableLocksPath.toString()); boolean locksExist = !lockedIds.isEmpty(); if (locksExist) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java index f6feafe9f5e..f6a71433228 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java @@ -180,7 +180,7 @@ public DistributedWorkQueue(String path, AccumuloConfiguration config, ServerCon this.context = context; this.timerInitialDelay = timerInitialDelay; this.timerPeriod = timerPeriod; - zoo = context.getZooReaderWriter(); + zoo = context.getZooSession().asReaderWriter(); } public ServerContext getContext() { diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java index f39645771f1..6daabba4e0a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java @@ -62,7 +62,7 @@ public static class ZooArbitrator implements Arbitrator { public ZooArbitrator(ServerContext context) { this.context = context; - rdr = context.getZooReader(); + rdr = context.getZooSession().asReader(); } @Override @@ -74,7 +74,7 @@ public boolean transactionAlive(String type, long tid) throws Exception { public static void start(ServerContext context, String type, long tid) throws KeeperException, InterruptedException { - ZooReaderWriter writer = context.getZooReaderWriter(); + ZooReaderWriter writer = context.getZooSession().asReaderWriter(); writer.putPersistentData(context.getZooKeeperRoot() + "/" + type, new byte[] {}, NodeExistsPolicy.OVERWRITE); writer.putPersistentData(context.getZooKeeperRoot() + "/" + type + "/" + tid, new byte[] {}, @@ -85,14 +85,14 @@ public static void start(ServerContext context, String type, long tid) public static void stop(ServerContext context, String type, long tid) throws KeeperException, InterruptedException { - ZooReaderWriter writer = context.getZooReaderWriter(); + ZooReaderWriter writer = context.getZooSession().asReaderWriter(); writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid, NodeMissingPolicy.SKIP); } public static void cleanup(ServerContext context, String type, long tid) throws KeeperException, InterruptedException { - ZooReaderWriter writer = context.getZooReaderWriter(); + ZooReaderWriter writer = context.getZooSession().asReaderWriter(); writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid, NodeMissingPolicy.SKIP); writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid + "-running", @@ -101,7 +101,7 @@ public static void cleanup(ServerContext context, String type, long tid) public static Set allTransactionsAlive(ServerContext context, String type) throws KeeperException, InterruptedException { - final ZooReader reader = context.getZooReaderWriter(); + final ZooReader reader = context.getZooSession().asReaderWriter(); final Set result = new HashSet<>(); final String parent = context.getZooKeeperRoot() + "/" + type; reader.sync(parent); diff --git a/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java b/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java index 338acf94860..46d3a7966b6 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java +++ b/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java @@ -23,13 +23,13 @@ import java.util.Properties; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.conf.store.PropStore; import org.easymock.EasyMock; @@ -47,22 +47,20 @@ public static ServerContext get() { return context; } - public static ServerContext getWithZK(InstanceId instanceID, String zk, int zkTimeout) { + public static ServerContext getWithMockZK(ZooSession zk) { var sc = get(); - expect(sc.getZooKeeperRoot()).andReturn(ZooUtil.getRoot(instanceID)).anyTimes(); - expect(sc.getInstanceID()).andReturn(instanceID).anyTimes(); - expect(sc.zkUserPath()).andReturn(ZooUtil.getRoot(instanceID) + Constants.ZUSERS).anyTimes(); - expect(sc.getZooKeepers()).andReturn(zk).anyTimes(); - expect(sc.getZooKeepersSessionTimeOut()).andReturn(zkTimeout).anyTimes(); + var zrw = new ZooReaderWriter(zk); + expect(sc.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReader()).andReturn(zrw).anyTimes(); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); return sc; } public static ServerContext getMockContextWithPropStore(final InstanceId instanceID, - ZooReaderWriter zrw, PropStore propStore) { + PropStore propStore) { ServerContext sc = createMock(ServerContext.class); expect(sc.getInstanceID()).andReturn(instanceID).anyTimes(); - expect(sc.getZooReaderWriter()).andReturn(zrw).anyTimes(); expect(sc.getZooKeeperRoot()).andReturn(ZooUtil.getRoot(instanceID)).anyTimes(); expect(sc.getPropStore()).andReturn(propStore).anyTimes(); return sc; diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/AccumuloConfigurationIsPropertySetTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/AccumuloConfigurationIsPropertySetTest.java index 3097ab6994c..755df592459 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/AccumuloConfigurationIsPropertySetTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/AccumuloConfigurationIsPropertySetTest.java @@ -98,7 +98,7 @@ public void setupMocks() { propStore.registerAsListener(anyObject(), anyObject()); expectLastCall().anyTimes(); - context = getMockContextWithPropStore(instanceId, null, propStore); + context = getMockContextWithPropStore(instanceId, propStore); SiteConfiguration siteConfig = SiteConfiguration.empty().build(); expect(context.getSiteConfiguration()).andReturn(siteConfig).anyTimes(); } diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java index 7fce935b7ee..2b57617b38c 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java @@ -41,7 +41,6 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.NamespaceId; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.NamespacePropKey; @@ -69,9 +68,7 @@ public void setUp() { propStore.registerAsListener(anyObject(), anyObject()); expectLastCall().anyTimes(); - ZooReaderWriter zrw = createMock(ZooReaderWriter.class); - - context = getMockContextWithPropStore(iid, zrw, propStore); + context = getMockContextWithPropStore(iid, propStore); parent = createMock(AccumuloConfiguration.class); reset(propStore); diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/SystemConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/SystemConfigurationTest.java index 17a8a561fa6..7249231b0b6 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/SystemConfigurationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/SystemConfigurationTest.java @@ -31,6 +31,7 @@ import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.reset; +import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -47,6 +48,7 @@ import org.apache.accumulo.server.conf.store.PropStore; import org.apache.accumulo.server.conf.store.SystemPropKey; import org.apache.accumulo.server.conf.store.impl.ZooPropStore; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -54,6 +56,7 @@ public class SystemConfigurationTest { private InstanceId instanceId; + private ServerContext context; private PropStore propStore; private SystemConfiguration sysConfig; @@ -61,7 +64,7 @@ public class SystemConfigurationTest { @BeforeEach public void initMocks() { instanceId = InstanceId.of(UUID.randomUUID()); - ServerContext context = createMock(ServerContext.class); + context = createMock(ServerContext.class); expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); propStore = createMock(ZooPropStore.class); expect(context.getPropStore()).andReturn(propStore).anyTimes(); @@ -78,7 +81,7 @@ public void initMocks() { VersionedProperties sysProps = new VersionedProperties(1, Instant.now(), Map.of(GC_PORT.getKey(), "1234", TSERV_SCAN_MAX_OPENFILES.getKey(), "19", TABLE_BLOOM_ENABLED.getKey(), "true")); - expect(propStore.get(eq(sysPropKey))).andReturn(sysProps).times(2); + expect(propStore.get(eq(sysPropKey))).andReturn(sysProps).once(); replay(propStore); ConfigurationCopy defaultConfig = @@ -88,6 +91,11 @@ public void initMocks() { sysConfig = new SystemConfiguration(context, sysPropKey, defaultConfig); } + @AfterEach + public void verifyMocks() { + verify(context, propStore); + } + @Test public void testFromDefault() { assertEquals("true", sysConfig.get(TABLE_BLOOM_ENABLED)); @@ -96,7 +104,6 @@ public void testFromDefault() { @Test public void testFromFixed() { - var sysPropKey = SystemPropKey.of(instanceId); assertEquals("9997", sysConfig.get(TSERV_CLIENTPORT)); // default @@ -132,6 +139,5 @@ public void testFromFixed() { assertTrue(sysConfig.isPropertySet(TSERV_SCAN_MAX_OPENFILES)); // fixed sys config assertTrue(sysConfig.isPropertySet(TABLE_BLOOM_ENABLED)); // sys config assertTrue(sysConfig.isPropertySet(TABLE_BLOOM_SIZE)); // default - } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java index ca67be50fb4..9cb055a5917 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java @@ -40,6 +40,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; @@ -61,6 +62,7 @@ public class PropStoreEventTest { // mocks private ServerContext context; + private ZooSession zk; private ZooReaderWriter zrw; private ReadyMonitor readyMonitor; @@ -68,8 +70,10 @@ public class PropStoreEventTest { public void initCommonMocks() throws Exception { instanceId = InstanceId.of(UUID.randomUUID()); context = createMock(ServerContext.class); + zk = createMock(ZooSession.class); zrw = createMock(ZooReaderWriter.class); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); + expect(context.getZooSession()).andReturn(zk).anyTimes(); expect(context.getZooKeepersSessionTimeOut()).andReturn(500).anyTimes(); expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); @@ -80,7 +84,7 @@ public void initCommonMocks() throws Exception { @AfterEach public void verifyMocks() { - verify(context, zrw, readyMonitor); + verify(context, zk, zrw, readyMonitor); } @Test @@ -96,9 +100,9 @@ public void zkChangeEventTest() throws Exception { readyMonitor.setReady(); expectLastCall().once(); - replay(context, zrw, readyMonitor, zkEvent); + replay(context, zk, zrw, readyMonitor, zkEvent); - PropStore propStore = new ZooPropStore(instanceId, zrw, readyMonitor, watcher, null); + PropStore propStore = new ZooPropStore(instanceId, zk, readyMonitor, watcher, null); StoreTestListener listener = new StoreTestListener(); propStore.registerAsListener(tablePropKey, listener); @@ -126,9 +130,9 @@ public void deleteEventTest() throws Exception { readyMonitor.setReady(); expectLastCall().once(); - replay(context, zrw, readyMonitor, zkEvent); + replay(context, zk, zrw, readyMonitor, zkEvent); - PropStore propStore = new ZooPropStore(instanceId, zrw, readyMonitor, watcher, null); + PropStore propStore = new ZooPropStore(instanceId, zk, readyMonitor, watcher, null); StoreTestListener listener = new StoreTestListener(); @@ -158,9 +162,9 @@ public void disconnectEventTest() throws Exception { readyMonitor.clearReady(); expectLastCall(); - replay(context, zrw, readyMonitor, zkEvent); + replay(context, zk, zrw, readyMonitor, zkEvent); - PropStore propStore = new ZooPropStore(instanceId, zrw, readyMonitor, watcher, null); + PropStore propStore = new ZooPropStore(instanceId, zk, readyMonitor, watcher, null); StoreTestListener listener = new StoreTestListener(); @@ -192,9 +196,9 @@ public void closedEventTest() throws Exception { readyMonitor.setClosed(); expectLastCall(); - replay(context, zrw, readyMonitor, zkEvent); + replay(context, zk, zrw, readyMonitor, zkEvent); - PropStore propStore = new ZooPropStore(instanceId, zrw, readyMonitor, watcher, null); + PropStore propStore = new ZooPropStore(instanceId, zk, readyMonitor, watcher, null); StoreTestListener listener = new StoreTestListener(); @@ -216,9 +220,9 @@ public void cacheChangeEventTest() throws Exception { readyMonitor.setReady(); expectLastCall().once(); - replay(context, zrw, readyMonitor); + replay(context, zk, zrw, readyMonitor); - PropStore propStore = new ZooPropStore(instanceId, zrw, readyMonitor, watcher, null); + PropStore propStore = new ZooPropStore(instanceId, zk, readyMonitor, watcher, null); StoreTestListener listener = new StoreTestListener(); @@ -253,9 +257,9 @@ public void validateWatcherSetTest() throws Exception { PropStoreWatcher watcher = new PropStoreWatcher(readyMonitor); - replay(context, zrw, readyMonitor); + replay(context, zk, zrw, readyMonitor); - ZooPropLoader loader = new ZooPropLoader(zrw, propCodec, watcher); + ZooPropLoader loader = new ZooPropLoader(zk, propCodec, watcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).build(); diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java index c47fc49752c..333535207ae 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java @@ -45,7 +45,7 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.PropStoreKey; @@ -62,7 +62,7 @@ public class ZooPropLoaderTest { private PropCacheCaffeineImplTest.TestTicker ticker; private InstanceId instanceId; - private ServerContext context; + private ZooSession zk; private PropStoreKey propStoreKey; private VersionedPropCodec propCodec; @@ -81,21 +81,21 @@ public void initCommonMocks() { propCodec = VersionedPropCodec.getDefault(); // mocks - context = createMock(ServerContext.class); - expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); - + zk = createMock(ZooSession.class); zrw = createMock(ZooReaderWriter.class); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); + replay(zk); propStoreWatcher = createMock(PropStoreWatcher.class); // loader used in tests - loader = new ZooPropLoader(zrw, propCodec, propStoreWatcher); + loader = new ZooPropLoader(zk, propCodec, propStoreWatcher); } @AfterEach public void verifyCommonMocks() { - verify(context, zrw, propStoreWatcher); + verify(zk, zrw, propStoreWatcher); } @Test @@ -114,7 +114,7 @@ public void loadTest() throws Exception { return (bytes); }).once(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); assertNotNull(loader.load(propStoreKey)); } @@ -147,7 +147,7 @@ public void loadAndCacheTest() throws Exception { return (bytes); }).once(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -175,7 +175,7 @@ public void loadFailTest() throws Exception { propStoreWatcher.signalZkChangeEvent(eq(propStoreKey)); expectLastCall(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -206,7 +206,7 @@ public void expireTest() throws Exception { return (bytes); }).times(2); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -253,7 +253,7 @@ public void reloadExceptionTest() throws Exception { propStoreWatcher.signalCacheChangeEvent(anyObject()); expectLastCall().anyTimes(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -275,7 +275,7 @@ public void reloadExceptionTest() throws Exception { @Test public void getIfCachedTest() { - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -316,7 +316,7 @@ public void removeTest() throws Exception { return (bytes); }).once(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -364,7 +364,7 @@ public void removeAllTest() throws Exception { return (bytes); }).once(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -382,7 +382,7 @@ public void removeAllTest() throws Exception { @Test public void getIfCachedNotPresentTest() { - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(loader).forTests(ticker).build(); @@ -413,7 +413,7 @@ public void captureExampleTest() throws Exception { return propCodec.toBytes(vProps); }).anyTimes(); - replay(context, zrw, propStoreWatcher); + replay(zrw, propStoreWatcher); Stat statCheck = new Stat(); statCheck.setVersion(9); diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java index a09f2bef8b5..c91e4e8b0bf 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java @@ -46,6 +46,7 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; @@ -65,15 +66,18 @@ public class ZooPropStoreTest { // mocks private ServerContext context; + private ZooSession zk; private ZooReaderWriter zrw; @BeforeEach public void init() throws Exception { instanceId = InstanceId.of(UUID.randomUUID()); context = createMock(ServerContext.class); + zk = createMock(ZooSession.class); zrw = createMock(ZooReaderWriter.class); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); - expect(zrw.getSessionTimeout()).andReturn(2_000).anyTimes(); + expect(context.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); + expect(zk.getSessionTimeout()).andReturn(2_000).anyTimes(); expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); expect(zrw.exists(eq(ZooUtil.getRoot(instanceId)), anyObject())).andReturn(true).anyTimes(); @@ -81,7 +85,7 @@ public void init() throws Exception { @AfterEach public void verifyMock() { - verify(context, zrw); + verify(context, zk, zrw); } @Test @@ -93,9 +97,9 @@ public void create() throws Exception { expect(zrw.putPrivatePersistentData(eq(propStoreKey.getPath()), capture(bytes), anyObject())) .andReturn(true).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); propStore.create(propStoreKey, Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M")); @@ -132,9 +136,9 @@ public void getTest() throws Exception { return propCodec.toBytes(vProps); }).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache. @@ -167,9 +171,9 @@ public void versionTest() throws Exception { return propCodec.toBytes(new VersionedProperties(props)); }).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); var vProps = propStore.get(propStoreKey); assertNotNull(vProps); assertEquals(expectedVersion, vProps.getDataVersion()); @@ -215,16 +219,14 @@ public void putAllTest() throws Exception { return true; }).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); Map updateProps = Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "4321", TABLE_SPLIT_THRESHOLD.getKey(), "123M"); propStore.putAll(propStoreKey, updateProps); - - verify(zrw); } @Test @@ -260,15 +262,14 @@ public void removeTest() throws Exception { return true; }).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); Set deleteNames = Set.of(TABLE_BULK_MAX_TABLETS.getKey(), TABLE_SPLIT_THRESHOLD.getKey()); propStore.removeProperties(propStoreKey, deleteNames); - verify(zrw); } @Test @@ -295,9 +296,9 @@ public void removeWithExceptionsTest() throws Exception { expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class))) .andThrow(new InterruptedException("mock forced interrupt exception")).once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); Set deleteNames = Set.of(TABLE_BULK_MAX_TABLETS.getKey(), TABLE_SPLIT_THRESHOLD.getKey()); @@ -345,12 +346,12 @@ public void validateWatcherSetTest() throws Exception { return propCodec.toBytes(new VersionedProperties(13, Instant.now(), props)); }).once(); - replay(context, zrw); + replay(context, zk, zrw); ReadyMonitor monitor = new TestReadyMonitor("testmon", 2000); PropStoreWatcher watcher = new TestWatcher(monitor); - ZooPropStore propStore = new ZooPropStore(instanceId, zrw, monitor, watcher, null); + ZooPropStore propStore = new ZooPropStore(instanceId, zk, monitor, watcher, null); assertNotNull(propStore.get(tablePropKey)); @@ -408,9 +409,9 @@ public void deleteTest() throws Exception { zrw.delete(eq(propStoreKey.getPath())); expectLastCall().once(); - replay(context, zrw); + replay(context, zk, zrw); - PropStore propStore = ZooPropStore.initialize(instanceId, zrw); + PropStore propStore = ZooPropStore.initialize(instanceId, zk); assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache. diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java index eddd4672107..bbaed9f4bf0 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java @@ -27,7 +27,9 @@ import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.isA; +import static org.easymock.EasyMock.isNull; import static org.easymock.EasyMock.newCapture; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; @@ -46,8 +48,8 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.MockServerContext; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; @@ -137,20 +139,20 @@ public void allOpts() { @Test public void instanceIdOutputTest() throws Exception { String uuid = UUID.randomUUID().toString(); - var context = MockServerContext.getWithZK(InstanceId.of(uuid), "fakeHost", 2000); - ZooReader zooReader = createMock(ZooReader.class); - expect(context.getZooReader()).andReturn(zooReader).anyTimes(); - + ZooSession zk = createMock(ZooSession.class); + var context = MockServerContext.getWithMockZK(zk); + expect(context.getInstanceID()).andReturn(InstanceId.of(uuid)).anyTimes(); var instanceName = "test"; - expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once(); - expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName))) + expect(zk.getChildren(eq(ZROOT + ZINSTANCES), isNull())).andReturn(List.of(instanceName)) + .once(); + expect(zk.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName), isNull(), isNull())) .andReturn(uuid.getBytes(UTF_8)).once(); - - String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt"; - context.close(); + expectLastCall().once(); - replay(context, zooReader); + replay(context, zk); + + String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt"; class ZooInfoViewerTestClazz extends ZooInfoViewer { @Override @@ -162,7 +164,7 @@ ServerContext getContext(ZooInfoViewer.Opts ots) { ZooInfoViewer viewer = new ZooInfoViewerTestClazz(); viewer.execute(new String[] {"--print-instances", "--outfile", testFileName}); - verify(zooReader, context); + verify(context, zk); String line; try (Scanner scanner = new Scanner(new File(testFileName))) { @@ -182,14 +184,14 @@ ServerContext getContext(ZooInfoViewer.Opts ots) { @Test public void instanceNameOutputTest() throws Exception { String uuid = UUID.randomUUID().toString(); - var context = MockServerContext.getWithZK(InstanceId.of(uuid), "fakeHost", 2000); - ZooReader zooReader = createMock(ZooReader.class); - expect(context.getZooReader()).andReturn(zooReader).anyTimes(); + ZooSession zk = createMock(ZooSession.class); + var context = MockServerContext.getWithMockZK(zk); var instanceName = "test"; - expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once(); - expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName))) + expect(zk.getChildren(eq(ZROOT + ZINSTANCES), isNull())).andReturn(List.of(instanceName)) + .once(); + expect(zk.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName), isNull(), isNull())) .andReturn(uuid.getBytes(UTF_8)).once(); - replay(context, zooReader); + replay(context, zk); String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt"; @@ -200,7 +202,7 @@ public void instanceNameOutputTest() throws Exception { ZooInfoViewer viewer = new ZooInfoViewer(); viewer.generateReport(context, opts); - verify(context, zooReader); + verify(context, zk); String line; try (Scanner scanner = new Scanner(new File(testFileName))) { @@ -223,26 +225,19 @@ public void instanceNameOutputTest() throws Exception { public void propTest() throws Exception { String uuid = UUID.randomUUID().toString(); InstanceId iid = InstanceId.of(uuid); - // ZooReaderWriter zrw = createMock(ZooReaderWriter.class); - // expect(zrw.getSessionTimeout()).andReturn(2_000).anyTimes(); - // expect(zrw.exists(eq("/accumulo/" + iid), anyObject())).andReturn(true).anyTimes(); - // replay(zrw); - - // ServerContext context = MockServerContext.getMockContextWithPropStore(iid, zrw, propStore); - - var context = MockServerContext.getWithZK(iid, "fakeHost", 2000); - ZooReader zooReader = createMock(ZooReader.class); - expect(context.getZooReader()).andReturn(zooReader).anyTimes(); + ZooSession zk = createMock(ZooSession.class); + var context = MockServerContext.getWithMockZK(zk); + expect(context.getInstanceID()).andReturn(iid).anyTimes(); var instanceName = "test"; - expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)) + expect(zk.getChildren(eq(ZROOT + ZINSTANCES), isNull())).andReturn(List.of(instanceName)) .anyTimes(); - expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName))) + expect(zk.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName), isNull(), isNull())) .andReturn(uuid.getBytes(UTF_8)).anyTimes(); var sysPropBytes = propCodec .toBytes(new VersionedProperties(123, Instant.now(), Map.of("s1", "sv1", "s2", "sv2"))); Capture sStat = newCapture(); - expect(zooReader.getData(eq(SystemPropKey.of(iid).getPath()), isA(PropStoreWatcher.class), + expect(zk.getData(eq(SystemPropKey.of(iid).getPath()), isA(PropStoreWatcher.class), capture(sStat))).andAnswer(() -> { Stat s = sStat.getValue(); s.setCtime(System.currentTimeMillis()); @@ -260,8 +255,8 @@ public void propTest() throws Exception { propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("n1", "nv1"))); NamespaceId nsId = NamespaceId.of("a"); Capture nsStat = newCapture(); - expect(zooReader.getData(eq(NamespacePropKey.of(iid, nsId).getPath()), - isA(PropStoreWatcher.class), capture(nsStat))).andAnswer(() -> { + expect(zk.getData(eq(NamespacePropKey.of(iid, nsId).getPath()), isA(PropStoreWatcher.class), + capture(nsStat))).andAnswer(() -> { Stat s = nsStat.getValue(); s.setCtime(System.currentTimeMillis()); s.setMtime(System.currentTimeMillis()); @@ -280,7 +275,7 @@ public void propTest() throws Exception { var tPropBytes = propCodec.toBytes(tProps); TableId tid = TableId.of("t"); Capture stat = newCapture(); - expect(zooReader.getData(eq(TablePropKey.of(iid, tid).getPath()), isA(PropStoreWatcher.class), + expect(zk.getData(eq(TablePropKey.of(iid, tid).getPath()), isA(PropStoreWatcher.class), capture(stat))).andAnswer(() -> { Stat s = stat.getValue(); s.setCtime(System.currentTimeMillis()); @@ -291,16 +286,18 @@ public void propTest() throws Exception { return tPropBytes; }).once(); - expect(zooReader.getData(tBasePath + "/t" + ZTABLE_NAMESPACE)) + expect(zk.getData(tBasePath + "/t" + ZTABLE_NAMESPACE, null, null)) .andReturn("+default".getBytes(UTF_8)).anyTimes(); + context.close(); + expectLastCall().once(); + + replay(context, zk); + NamespacePropKey nsKey = NamespacePropKey.of(iid, nsId); log.trace("namespace base path: {}", nsKey.getPath()); String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt"; - context.close(); - - replay(context, zooReader); class ZooInfoViewerTestClazz extends ZooInfoViewer { @Override @@ -312,7 +309,7 @@ ServerContext getContext(ZooInfoViewer.Opts ots) { ZooInfoViewer viewer = new ZooInfoViewerTestClazz(); viewer.execute(new String[] {"--print-props", "--outfile", testFileName}); - verify(zooReader, context); + verify(context, zk); Map props = new HashMap<>(); try (Scanner scanner = new Scanner(new File(testFileName))) { diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java deleted file mode 100644 index e77832ceabc..00000000000 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java +++ /dev/null @@ -1,68 +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.server.conf.util; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.core.Constants.ZINSTANCES; -import static org.apache.accumulo.core.Constants.ZROOT; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import org.apache.accumulo.core.data.InstanceId; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class ZooPropUtilsTest { - private static final Logger LOG = LoggerFactory.getLogger(ZooPropUtilsTest.class); - - @Test - public void fetchInstancesFromZk() throws Exception { - - String instAName = "INST_A"; - InstanceId instA = InstanceId.of(UUID.randomUUID()); - String instBName = "INST_B"; - InstanceId instB = InstanceId.of(UUID.randomUUID()); - - ZooReader zooReader = createMock(ZooReader.class); - String namePath = ZROOT + ZINSTANCES; - expect(zooReader.getChildren(eq(namePath))).andReturn(List.of(instAName, instBName)).once(); - expect(zooReader.getData(eq(namePath + "/" + instAName))) - .andReturn(instA.canonical().getBytes(UTF_8)).once(); - expect(zooReader.getData(eq(namePath + "/" + instBName))) - .andReturn(instB.canonical().getBytes(UTF_8)).once(); - replay(zooReader); - - Map instanceMap = ZooPropUtils.readInstancesFromZk(zooReader); - - LOG.trace("id map returned: {}", instanceMap); - assertEquals(Map.of(instAName, instA, instBName, instB), instanceMap); - verify(zooReader); - } - -} diff --git a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java index f56510394ba..c0b4817a2f9 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java @@ -31,9 +31,11 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.conf.SiteConfiguration; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.zookeeper.data.Stat; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -45,7 +47,8 @@ public class InitializeTest { private Configuration conf; private VolumeManager fs; private SiteConfiguration sconf; - private ZooReaderWriter zoo; + private ZooSession zk; + private ZooReaderWriter zrw; private InitialConfiguration initConfig; @BeforeEach @@ -59,18 +62,19 @@ public void setUp() { expect(sconf.get(Property.INSTANCE_SECRET)) .andReturn(Property.INSTANCE_SECRET.getDefaultValue()).anyTimes(); expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1").anyTimes(); - zoo = createMock(ZooReaderWriter.class); + zk = createMock(ZooSession.class); + zrw = new ZooReaderWriter(zk); } @AfterEach public void tearDown() { - verify(sconf, zoo, fs); + verify(sconf, zk, fs); } @Test public void testIsInitialized_HasInstanceId() throws Exception { expect(fs.exists(anyObject(Path.class))).andReturn(true); - replay(sconf, zoo, fs); + replay(sconf, zk, fs); assertTrue(Initialize.isInitialized(fs, initConfig)); } @@ -78,40 +82,40 @@ public void testIsInitialized_HasInstanceId() throws Exception { public void testIsInitialized_HasDataVersion() throws Exception { expect(fs.exists(anyObject(Path.class))).andReturn(false); expect(fs.exists(anyObject(Path.class))).andReturn(true); - replay(sconf, zoo, fs); + replay(sconf, zk, fs); assertTrue(Initialize.isInitialized(fs, initConfig)); } @Test public void testCheckInit_NoZK() throws Exception { - expect(zoo.exists("/")).andReturn(false); - replay(sconf, zoo, fs); - assertThrows(IllegalStateException.class, () -> Initialize.checkInit(zoo, fs, initConfig)); + expect(zk.exists("/", null)).andReturn(null); + replay(sconf, zk, fs); + assertThrows(IllegalStateException.class, () -> Initialize.checkInit(zrw, fs, initConfig)); } @Test public void testCheckInit_AlreadyInit() throws Exception { - expect(zoo.exists("/")).andReturn(true); + expect(zk.exists("/", null)).andReturn(new Stat()); expect(fs.exists(anyObject(Path.class))).andReturn(true); - replay(sconf, zoo, fs); - assertThrows(IOException.class, () -> Initialize.checkInit(zoo, fs, initConfig)); + replay(sconf, zk, fs); + assertThrows(IOException.class, () -> Initialize.checkInit(zrw, fs, initConfig)); } @Test public void testCheckInit_FSException() throws Exception { - expect(zoo.exists("/")).andReturn(true); + expect(zk.exists("/", null)).andReturn(new Stat()); expect(fs.exists(anyObject(Path.class))).andThrow(new IOException()); - replay(sconf, zoo, fs); - assertThrows(IOException.class, () -> Initialize.checkInit(zoo, fs, initConfig)); + replay(sconf, zk, fs); + assertThrows(IOException.class, () -> Initialize.checkInit(zrw, fs, initConfig)); } @Test public void testCheckInit_OK() throws Exception { - expect(zoo.exists("/")).andReturn(true); + expect(zk.exists("/", null)).andReturn(new Stat()); // check for volumes initialized calls exists twice for each volume // once for instance_id, and once for version expect(fs.exists(anyObject(Path.class))).andReturn(false).times(4); - replay(sconf, zoo, fs); - Initialize.checkInit(zoo, fs, initConfig); + replay(sconf, zk, fs); + Initialize.checkInit(zrw, fs, initConfig); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java index 2cb74d2b9bd..25c7003cf8e 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java @@ -46,6 +46,7 @@ import org.apache.accumulo.core.metrics.MetricsInfo; import org.apache.accumulo.core.metrics.MetricsProducer; import org.apache.accumulo.core.trace.TraceUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.client.ClientServiceHandler; import org.apache.thrift.server.TServer; @@ -58,14 +59,17 @@ public class TServerUtilsTest { private ServerContext context; + private ZooSession zk; private MetricsInfo metricsInfo; private final ConfigurationCopy conf = new ConfigurationCopy(DefaultConfiguration.getInstance()); @BeforeEach public void createMockServerContext() { context = createMock(ServerContext.class); - expect(context.getZooReader()).andReturn(null).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(null).anyTimes(); + zk = createMock(ZooSession.class); + expect(context.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReader()).andReturn(null).anyTimes(); + expect(zk.asReaderWriter()).andReturn(null).anyTimes(); expect(context.getProperties()).andReturn(new Properties()).anyTimes(); expect(context.getZooKeepers()).andReturn("").anyTimes(); expect(context.getInstanceName()).andReturn("instance").anyTimes(); diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributorTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributorTest.java index a2653513706..59ded20cacb 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributorTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyDistributorTest.java @@ -41,9 +41,11 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException.AuthFailedException; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -62,103 +64,96 @@ public static void setupKeyGenerator() throws Exception { keyGen.init(KEY_LENGTH); } + private ZooSession zk; private ZooReaderWriter zrw; private String baseNode = Constants.ZDELEGATION_TOKEN_KEYS; @BeforeEach public void setupMocks() { zrw = createMock(ZooReaderWriter.class); + zk = createMock(ZooSession.class); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); + } + + @AfterEach + public void verifyMocks() { + verify(zk, zrw); } @Test public void testInitialize() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); - // Attempt to create the directory and fail expect(zrw.exists(baseNode)).andReturn(false); - expect( - zrw.putPrivatePersistentData(eq(baseNode), aryEq(new byte[0]), eq(NodeExistsPolicy.FAIL))) + expect(zrw.putPrivatePersistentData(baseNode, new byte[0], NodeExistsPolicy.FAIL)) .andThrow(new AuthFailedException()); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(AuthFailedException.class, distributor::initialize); - - verify(zrw); } @Test public void testInitializeCreatesParentNode() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); - // Attempt to create the directory and fail expect(zrw.exists(baseNode)).andReturn(false); expect(zrw.putPrivatePersistentData(eq(baseNode), anyObject(), eq(NodeExistsPolicy.FAIL))) .andReturn(true); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); - - verify(zrw); } @Test public void testInitializedNotCalledAdvertise() { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(IllegalStateException.class, () -> distributor.advertise(new AuthenticationKey(1, 0L, 5L, keyGen.generateKey()))); } @Test public void testInitializedNotCalledCurrentKeys() { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(IllegalStateException.class, distributor::getCurrentKeys); } @Test public void testInitializedNotCalledRemove() { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(IllegalStateException.class, () -> distributor.remove(new AuthenticationKey(1, 0L, 5L, keyGen.generateKey()))); } @Test public void testMissingAcl() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); // Attempt to create the directory and fail expect(zrw.exists(baseNode)).andReturn(true); expect(zrw.getACL(eq(baseNode))).andReturn(Collections.emptyList()); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(IllegalStateException.class, distributor::initialize); - verify(zrw); } @Test public void testBadAcl() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); // Attempt to create the directory and fail expect(zrw.exists(baseNode)).andReturn(true); expect(zrw.getACL(eq(baseNode))).andReturn(Collections.singletonList( new ACL(ZooUtil.PRIVATE.get(0).getPerms(), new Id("digest", "somethingweird")))); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); assertThrows(IllegalStateException.class, distributor::initialize); - verify(zrw); } @Test public void testAdvertiseKey() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); AuthenticationKey key = new AuthenticationKey(1, 0L, 10L, keyGen.generateKey()); ByteArrayOutputStream baos = new ByteArrayOutputStream(); key.write(new DataOutputStream(baos)); @@ -173,18 +168,15 @@ public void testAdvertiseKey() throws Exception { expect(zrw.putPrivatePersistentData(eq(path), aryEq(serialized), eq(NodeExistsPolicy.FAIL))) .andReturn(true); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); distributor.advertise(key); - - verify(zrw); } @Test public void testAlreadyAdvertisedKey() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); AuthenticationKey key = new AuthenticationKey(1, 0L, 10L, keyGen.generateKey()); String path = baseNode + "/" + key.getKeyId(); @@ -194,18 +186,15 @@ public void testAlreadyAdvertisedKey() throws Exception { new ACL(ZooUtil.PRIVATE.get(0).getPerms(), new Id("digest", "accumulo:DEFAULT")))); expect(zrw.exists(path)).andReturn(true); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); distributor.advertise(key); - - verify(zrw); } @Test public void testRemoveKey() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); AuthenticationKey key = new AuthenticationKey(1, 0L, 10L, keyGen.generateKey()); String path = baseNode + "/" + key.getKeyId(); @@ -217,18 +206,15 @@ public void testRemoveKey() throws Exception { zrw.delete(path); expectLastCall().once(); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); distributor.remove(key); - - verify(zrw); } @Test public void testRemoveMissingKey() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); AuthenticationKey key = new AuthenticationKey(1, 0L, 10L, keyGen.generateKey()); String path = baseNode + "/" + key.getKeyId(); @@ -238,18 +224,15 @@ public void testRemoveMissingKey() throws Exception { new ACL(ZooUtil.PRIVATE.get(0).getPerms(), new Id("digest", "accumulo:DEFAULT")))); expect(zrw.exists(path)).andReturn(false); - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); distributor.remove(key); - - verify(zrw); } @Test public void testGetCurrentKeys() throws Exception { - ZooAuthenticationKeyDistributor distributor = - new ZooAuthenticationKeyDistributor(zrw, baseNode); List keys = new ArrayList<>(5); List serializedKeys = new ArrayList<>(5); List children = new ArrayList<>(5); @@ -270,11 +253,10 @@ public void testGetCurrentKeys() throws Exception { expect(zrw.getData(baseNode + "/" + i)).andReturn(serializedKeys.get(i - 1)); } - replay(zrw); + replay(zk, zrw); + var distributor = new ZooAuthenticationKeyDistributor(zk, baseNode); distributor.initialize(); assertEquals(keys, distributor.getCurrentKeys()); - - verify(zrw); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java index df71d4fe8c0..fba25de8e9b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java @@ -43,10 +43,13 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.data.Stat; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -65,7 +68,7 @@ public static void setupKeyGenerator() throws Exception { keyGen.init(KEY_LENGTH); } - private ZooReader zk; + private ZooSession zk; private InstanceId instanceId; private String baseNode; private long tokenLifetime = DAYS.toMillis(7); @@ -74,11 +77,20 @@ public static void setupKeyGenerator() throws Exception { @BeforeEach public void setupMocks() { - zk = createMock(ZooReader.class); + zk = createMock(ZooSession.class); instanceId = InstanceId.of(UUID.randomUUID()); baseNode = ZooUtil.getRoot(instanceId) + Constants.ZDELEGATION_TOKEN_KEYS; secretManager = new AuthenticationTokenSecretManager(instanceId, tokenLifetime); + + expect(zk.asReader()).andReturn(new ZooReader(zk)).once(); + replay(zk); keyWatcher = new ZooAuthenticationKeyWatcher(secretManager, zk, baseNode); + reset(zk); + } + + @AfterEach + public void verifyMocks() { + verify(zk); } @Test @@ -90,7 +102,6 @@ public void testBaseNodeCreated() throws Exception { keyWatcher.process(event); - verify(zk); assertTrue(secretManager.getKeys().isEmpty()); } @@ -103,13 +114,12 @@ public void testBaseNodeCreatedWithChildren() throws Exception { List children = Arrays.asList("1", "2"); expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); - expect(zk.getData(baseNode + "/1", keyWatcher)).andReturn(serializedKey1); - expect(zk.getData(baseNode + "/2", keyWatcher)).andReturn(serializedKey2); + expect(zk.getData(baseNode + "/1", keyWatcher, null)).andReturn(serializedKey1); + expect(zk.getData(baseNode + "/2", keyWatcher, null)).andReturn(serializedKey2); replay(zk); keyWatcher.process(event); - verify(zk); assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); @@ -124,13 +134,12 @@ public void testBaseNodeChildrenChanged() throws Exception { List children = Arrays.asList("1", "2"); expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); - expect(zk.getData(baseNode + "/1", keyWatcher)).andReturn(serializedKey1); - expect(zk.getData(baseNode + "/2", keyWatcher)).andReturn(serializedKey2); + expect(zk.getData(baseNode + "/1", keyWatcher, null)).andReturn(serializedKey1); + expect(zk.getData(baseNode + "/2", keyWatcher, null)).andReturn(serializedKey2); replay(zk); keyWatcher.process(event); - verify(zk); assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); @@ -150,7 +159,6 @@ public void testBaseNodeDeleted() { keyWatcher.process(event); - verify(zk); assertEquals(0, secretManager.getKeys().size()); assertFalse(secretManager.isCurrentKeySet()); } @@ -163,7 +171,6 @@ public void testBaseNodeDataChanged() { keyWatcher.process(event); - verify(zk); assertEquals(0, secretManager.getKeys().size()); assertFalse(secretManager.isCurrentKeySet()); } @@ -177,12 +184,11 @@ public void testChildChanged() throws Exception { assertEquals(1, secretManager.getKeys().size()); byte[] serializedKey2 = serialize(key2); - expect(zk.getData(event.getPath(), keyWatcher)).andReturn(serializedKey2); + expect(zk.getData(event.getPath(), keyWatcher, null)).andReturn(serializedKey2); replay(zk); keyWatcher.process(event); - verify(zk); assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); @@ -202,7 +208,6 @@ public void testChildDeleted() { keyWatcher.process(event); - verify(zk); assertEquals(1, secretManager.getKeys().size()); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); assertEquals(key2, secretManager.getCurrentKey()); @@ -222,7 +227,6 @@ public void testChildChildrenChanged() { // Does nothing keyWatcher.process(event); - verify(zk); assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); @@ -231,13 +235,12 @@ public void testChildChildrenChanged() { @Test public void testInitialUpdateNoNode() throws Exception { - expect(zk.exists(baseNode, keyWatcher)).andReturn(false); + expect(zk.exists(baseNode, keyWatcher)).andReturn(null); replay(zk); keyWatcher.updateAuthKeys(); - verify(zk); assertEquals(0, secretManager.getKeys().size()); assertNull(secretManager.getCurrentKey()); } @@ -248,17 +251,17 @@ public void testInitialUpdateWithKeys() throws Exception { AuthenticationKey key1 = new AuthenticationKey(1, 0L, 10000L, keyGen.generateKey()), key2 = new AuthenticationKey(5, key1.getExpirationDate(), 20000L, keyGen.generateKey()); - expect(zk.exists(baseNode, keyWatcher)).andReturn(true); + expect(zk.exists(baseNode, keyWatcher)).andReturn(new Stat()); expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); - expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher)).andReturn(serialize(key1)); - expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher)).andReturn(serialize(key2)); + expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key1)); + expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key2)); replay(zk); keyWatcher.updateAuthKeys(); - verify(zk); - assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); assertEquals(key2, secretManager.getKeys().get(key2.getKeyId())); @@ -283,10 +286,12 @@ private void lostZooKeeperBase(WatchedEvent disconnectEvent, WatchedEvent reconn AuthenticationKey key1 = new AuthenticationKey(1, 0L, 10000L, keyGen.generateKey()), key2 = new AuthenticationKey(5, key1.getExpirationDate(), 20000L, keyGen.generateKey()); - expect(zk.exists(baseNode, keyWatcher)).andReturn(true); + expect(zk.exists(baseNode, keyWatcher)).andReturn(new Stat()); expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); - expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher)).andReturn(serialize(key1)); - expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher)).andReturn(serialize(key2)); + expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key1)); + expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key2)); replay(zk); @@ -294,8 +299,6 @@ private void lostZooKeeperBase(WatchedEvent disconnectEvent, WatchedEvent reconn keyWatcher.updateAuthKeys(); keyWatcher.process(disconnectEvent); - verify(zk); - // We should have no auth keys when we're disconnected assertEquals(0, secretManager.getKeys().size(), "Secret manager should be empty after a disconnect"); @@ -303,18 +306,18 @@ private void lostZooKeeperBase(WatchedEvent disconnectEvent, WatchedEvent reconn reset(zk); - expect(zk.exists(baseNode, keyWatcher)).andReturn(true); + expect(zk.exists(baseNode, keyWatcher)).andReturn(new Stat()); expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); - expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher)).andReturn(serialize(key1)); - expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher)).andReturn(serialize(key2)); + expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key1)); + expect(zk.getData(baseNode + "/" + key2.getKeyId(), keyWatcher, null)) + .andReturn(serialize(key2)); replay(zk); // Reconnect again, get all the keys keyWatcher.process(reconnectEvent); - verify(zk); - // Verify we have both keys assertEquals(2, secretManager.getKeys().size()); assertEquals(key1, secretManager.getKeys().get(key1.getKeyId())); @@ -326,11 +329,11 @@ public void missingKeyAfterGetChildren() throws Exception { List children = Arrays.asList("1"); AuthenticationKey key1 = new AuthenticationKey(1, 0L, 10000L, keyGen.generateKey()); - expect(zk.exists(baseNode, keyWatcher)).andReturn(true); + expect(zk.exists(baseNode, keyWatcher)).andReturn(new Stat()); // We saw key1 expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children); // but it was gone when we tried to access it (manager deleted it) - expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher)) + expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher, null)) .andThrow(new NoNodeException()); replay(zk); @@ -338,8 +341,6 @@ public void missingKeyAfterGetChildren() throws Exception { // Initialize keyWatcher.updateAuthKeys(); - verify(zk); - // We should have no auth keys after initializing things assertEquals(0, secretManager.getKeys().size(), "Secret manager should be empty after a disconnect"); diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java index 6621feb93ff..a3f91a7cf82 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java @@ -39,16 +39,15 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.data.InstanceId; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.util.ByteArraySet; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.MockServerContext; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.junit.jupiter.api.Test; @@ -143,17 +142,16 @@ public void testUserAuthentication() throws Exception { // mocking zk interaction var instanceId = InstanceId.of("example"); - ServerContext context = MockServerContext.getWithZK(instanceId, "", 30_000); - ZooReaderWriter zr = createMock(ZooReaderWriter.class); - expect(context.getZooReader()).andReturn(zr).anyTimes(); - ZooKeeper zk = createMock(ZooKeeper.class); + ZooSession zk = createMock(ZooSession.class); + ServerContext context = MockServerContext.getWithMockZK(zk); + expect(context.zkUserPath()).andReturn(ZooUtil.getRoot(instanceId) + Constants.ZUSERS) + .anyTimes(); expect(zk.getChildren(anyObject(), anyObject())).andReturn(Arrays.asList(principal)).anyTimes(); expect(zk.exists(matches(ZooUtil.getRoot(instanceId) + Constants.ZUSERS + "/" + principal), anyObject(Watcher.class))).andReturn(new Stat()).anyTimes(); - expect(zr.getZooKeeper()).andReturn(zk).anyTimes(); expect(zk.getData(matches(ZooUtil.getRoot(instanceId) + Constants.ZUSERS + "/" + principal), anyObject(), anyObject())).andReturn(newHash).once(); - replay(context, zr, zk); + replay(context, zk); // creating authenticator ZKAuthenticator auth = new ZKAuthenticator(); @@ -162,6 +160,6 @@ public void testUserAuthentication() throws Exception { PasswordToken token = new PasswordToken(rawPass.clone()); // verifying that if the new type of hash is stored in zk authentication works as expected assertTrue(auth.authenticateUser(principal, token)); - verify(context, zr, zk); + verify(context, zk); } } diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java index 5a8bb11b775..3f885eec270 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java @@ -21,7 +21,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.Constants.ZGC_LOCK; import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; @@ -39,7 +38,7 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.util.serviceStatus.ServiceStatusReport; import org.apache.accumulo.server.util.serviceStatus.StatusSummary; import org.apache.zookeeper.KeeperException; @@ -53,28 +52,19 @@ public class ServiceStatusCmdTest { private static final Logger LOG = LoggerFactory.getLogger(ServiceStatusCmdTest.class); - private ServerContext context; - private String zRoot; + private final String zRoot = ZooUtil.getRoot(InstanceId.of(UUID.randomUUID())); + private ZooSession zk; private ZooReader zooReader; @BeforeEach public void populateContext() { - InstanceId iid = InstanceId.of(UUID.randomUUID()); - zRoot = ZooUtil.getRoot(iid); - context = createMock(ServerContext.class); - expect(context.getInstanceID()).andReturn(iid).anyTimes(); - expect(context.getZooKeeperRoot()).andReturn(zRoot).anyTimes(); - - zooReader = createMock(ZooReader.class); - - expect(context.getZooReader()).andReturn(zooReader).anyTimes(); - - replay(context); + zk = createMock(ZooSession.class); + zooReader = new ZooReader(zk); } @AfterEach public void validateMocks() { - verify(context, zooReader); + verify(zk); } @Test @@ -91,16 +81,16 @@ void testManagerHosts() throws Exception { "{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"hostA:9999\",\"group\":\"manager1\"}]}"; String lockPath = zRoot + Constants.ZMANAGER_LOCK; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, lock2Name, lock3Name)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name, lock3Name)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))).andReturn(lock1data.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock1Name, null, null)).andReturn(lock1data.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lock2Data.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock2Name, null, null)).andReturn(lock2Data.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock3Name))).andReturn(lock3Data.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock3Name, null, null)).andReturn(lock3Data.getBytes(UTF_8)) .anyTimes(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getManagerStatus(zooReader, zRoot); @@ -136,13 +126,13 @@ void testMonitorHosts() throws Exception { "{\"descriptors\":[{\"uuid\":\"87465459-9c8f-4f95-b4c6-ef3029030d05\",\"service\":\"NONE\",\"address\":\"hostB\",\"group\":\"default\"}]}"; String lockPath = zRoot + Constants.ZMONITOR_LOCK; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, lock2Name)).anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))).andReturn(host1.getBytes(UTF_8)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)).anyTimes(); + expect(zk.getData(lockPath + "/" + lock1Name, null, null)).andReturn(host1.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(host2.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock2Name, null, null)).andReturn(host2.getBytes(UTF_8)) .anyTimes(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getMonitorStatus(zooReader, zRoot); @@ -211,21 +201,21 @@ void testTServerHosts() throws Exception { + host3 + "\",\"group\":\"default\"}]}"; String basePath = zRoot + Constants.ZTSERVERS; - expect(zooReader.getChildren(eq(basePath))).andReturn(List.of(host1, host2, host3)).anyTimes(); + expect(zk.getChildren(basePath, null)).andReturn(List.of(host1, host2, host3)).anyTimes(); - expect(zooReader.getChildren(eq(basePath + "/" + host1))).andReturn(List.of(lock1Name)).once(); - expect(zooReader.getData(eq(basePath + "/" + host1 + "/" + lock1Name))) + expect(zk.getChildren(basePath + "/" + host1, null)).andReturn(List.of(lock1Name)).once(); + expect(zk.getData(basePath + "/" + host1 + "/" + lock1Name, null, null)) .andReturn(lockData1.getBytes(UTF_8)).anyTimes(); - expect(zooReader.getChildren(eq(basePath + "/" + host2))).andReturn(List.of(lock2Name)).once(); - expect(zooReader.getData(eq(basePath + "/" + host2 + "/" + lock2Name))) + expect(zk.getChildren(basePath + "/" + host2, null)).andReturn(List.of(lock2Name)).once(); + expect(zk.getData(basePath + "/" + host2 + "/" + lock2Name, null, null)) .andReturn(lockData2.getBytes(UTF_8)).anyTimes(); - expect(zooReader.getChildren(eq(basePath + "/" + host3))).andReturn(List.of(lock3Name)).once(); - expect(zooReader.getData(eq(basePath + "/" + host3 + "/" + lock3Name))) + expect(zk.getChildren(basePath + "/" + host3, null)).andReturn(List.of(lock3Name)).once(); + expect(zk.getData(basePath + "/" + host3 + "/" + lock3Name, null, null)) .andReturn(lockData3.getBytes(UTF_8)).anyTimes(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getTServerStatus(zooReader, zRoot); @@ -289,26 +279,26 @@ void testScanServerHosts() throws Exception { + host4 + "\",\"group\":\"default\"}]}"; String lockPath = zRoot + Constants.ZSSERVERS; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(host1, host2, host3, host4)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(host1, host2, host3, host4)) .anyTimes(); - expect(zooReader.getChildren(eq(lockPath + "/" + host1))).andReturn(List.of(lock1Name)).once(); - expect(zooReader.getData(eq(lockPath + "/" + host1 + "/" + lock1Name))) + expect(zk.getChildren(lockPath + "/" + host1, null)).andReturn(List.of(lock1Name)).once(); + expect(zk.getData(lockPath + "/" + host1 + "/" + lock1Name, null, null)) .andReturn(lockData1.getBytes(UTF_8)).once(); - expect(zooReader.getChildren(eq(lockPath + "/" + host2))).andReturn(List.of(lock2Name)).once(); - expect(zooReader.getData(eq(lockPath + "/" + host2 + "/" + lock2Name))) + expect(zk.getChildren(lockPath + "/" + host2, null)).andReturn(List.of(lock2Name)).once(); + expect(zk.getData(lockPath + "/" + host2 + "/" + lock2Name, null, null)) .andReturn(lockData2.getBytes(UTF_8)).once(); - expect(zooReader.getChildren(eq(lockPath + "/" + host3))).andReturn(List.of(lock3Name)).once(); - expect(zooReader.getData(eq(lockPath + "/" + host3 + "/" + lock3Name))) + expect(zk.getChildren(lockPath + "/" + host3, null)).andReturn(List.of(lock3Name)).once(); + expect(zk.getData(lockPath + "/" + host3 + "/" + lock3Name, null, null)) .andReturn(lockData3.getBytes(UTF_8)).once(); - expect(zooReader.getChildren(eq(lockPath + "/" + host4))).andReturn(List.of(lock4Name)).once(); - expect(zooReader.getData(eq(lockPath + "/" + host4 + "/" + lock4Name))) + expect(zk.getChildren(lockPath + "/" + host4, null)).andReturn(List.of(lock4Name)).once(); + expect(zk.getData(lockPath + "/" + host4 + "/" + lock4Name, null, null)) .andReturn(lockData4.getBytes(UTF_8)).once(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getScanServerStatus(zooReader, zRoot); @@ -346,16 +336,16 @@ void testCoordinatorHosts() throws Exception { + host3 + "\",\"group\":\"coord2\"}]}\n"; String lockPath = zRoot + Constants.ZCOORDINATOR_LOCK; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, lock2Name, lock3Name)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name, lock3Name)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))).andReturn(lockData1.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock1Name, null, null)).andReturn(lockData1.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lockData2.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock2Name, null, null)).andReturn(lockData2.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock3Name))).andReturn(lockData3.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock3Name, null, null)).andReturn(lockData3.getBytes(UTF_8)) .anyTimes(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getCoordinatorStatus(zooReader, zRoot); @@ -384,14 +374,14 @@ void testCoordinatorHosts() throws Exception { @Test public void testCompactorStatus() throws Exception { String lockPath = zRoot + Constants.ZCOMPACTORS; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of("q1", "q2")).once(); + expect(zk.getChildren(lockPath, null)).andReturn(List.of("q1", "q2")).once(); - expect(zooReader.getChildren(eq(lockPath + "/q1"))) - .andReturn(List.of("hostA:8080", "hostC:8081")).once(); - expect(zooReader.getChildren(eq(lockPath + "/q2"))) - .andReturn(List.of("hostB:9090", "hostD:9091")).once(); + expect(zk.getChildren(lockPath + "/q1", null)).andReturn(List.of("hostA:8080", "hostC:8081")) + .once(); + expect(zk.getChildren(lockPath + "/q2", null)).andReturn(List.of("hostB:9090", "hostD:9091")) + .once(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getCompactorStatus(zooReader, zRoot); @@ -418,13 +408,13 @@ public void testGcHosts() throws Exception { "{\"descriptors\":[{\"uuid\":\"5c901352-b027-4f78-8ee1-05ae163fbb0e\",\"service\":\"GC\",\"address\":\"" + host1 + "\",\"group\":\"gc1\"}]}"; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, lock2Name)).once(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))).andReturn(lockData1.getBytes(UTF_8)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)).once(); + expect(zk.getData(lockPath + "/" + lock1Name, null, null)).andReturn(lockData1.getBytes(UTF_8)) .once(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lockData2.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock2Name, null, null)).andReturn(lockData2.getBytes(UTF_8)) .once(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getGcStatus(zooReader, zRoot); @@ -461,18 +451,16 @@ void zkNodeDeletedTest() throws Exception { + host3 + "\",\"group\":\"manager1\"}]}"; String lockPath = zRoot + Constants.ZMANAGER_LOCK; - expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, lock2Name, lock3Name)) + expect(zk.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name, lock3Name)) .anyTimes(); - // expect(zooReader.getData(eq(lockPath + "/" + lock1Name))) - // .andThrow(new KeeperException.NoNodeException("no node forced exception")).once(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))) + expect(zk.getData(lockPath + "/" + lock1Name, null, null)) .andThrow(new KeeperException.NoNodeException("no node forced exception")).once(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lock2Data.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock2Name, null, null)).andReturn(lock2Data.getBytes(UTF_8)) .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock3Name))).andReturn(lock3Data.getBytes(UTF_8)) + expect(zk.getData(lockPath + "/" + lock3Name, null, null)).andReturn(lock3Data.getBytes(UTF_8)) .anyTimes(); - replay(zooReader); + replay(zk); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getManagerStatus(zooReader, zRoot); @@ -490,7 +478,7 @@ void zkNodeDeletedTest() throws Exception { @Test public void testServiceStatusCommandOpts() { - replay(zooReader); // needed for @AfterAll verify + replay(zk); ServiceStatusCmd.Opts opts = new ServiceStatusCmd.Opts(); assertFalse(opts.json); assertFalse(opts.noHosts); diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java index 09d1852f466..70e024022bd 100644 --- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java +++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java @@ -216,8 +216,8 @@ protected void getCoordinatorLock(HostAndPort clientAddress) final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK; final UUID zooLockUUID = UUID.randomUUID(); - coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), - ServiceLock.path(lockPath), zooLockUUID); + coordinatorLock = + new ServiceLock(getContext().getZooSession(), ServiceLock.path(lockPath), zooLockUUID); while (true) { HAServiceLockWatcher coordinatorLockWatcher = new HAServiceLockWatcher("coordinator"); @@ -733,7 +733,7 @@ private void deleteEmpty(ZooReaderWriter zoorw, String path) private void cleanUpCompactors() { final String compactorQueuesPath = getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS; - var zoorw = getContext().getZooReaderWriter(); + var zoorw = getContext().getZooSession().asReaderWriter(); try { var queues = zoorw.getChildren(compactorQueuesPath); diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java index 3d60d89fb19..f93440ef521 100644 --- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java +++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java @@ -254,7 +254,7 @@ protected void announceExistence(HostAndPort clientAddress) String hostPort = ExternalCompactionUtil.getHostPortString(clientAddress); - ZooReaderWriter zoo = getContext().getZooReaderWriter(); + ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); String compactorQueuePath = getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + this.queueName; String zPath = compactorQueuePath + "/" + hostPort; @@ -262,16 +262,13 @@ protected void announceExistence(HostAndPort clientAddress) try { zoo.mkdirs(compactorQueuePath); zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP); - } catch (KeeperException e) { - if (e.code() == KeeperException.Code.NOAUTH) { - LOG.error("Failed to write to ZooKeeper. Ensure that" - + " accumulo.properties, specifically instance.secret, is consistent."); - } - throw e; + } catch (KeeperException.NoAuthException e) { + LOG.error("Failed to write to ZooKeeper. Ensure that" + + " accumulo.properties, specifically instance.secret, is consistent."); } - compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), - ServiceLock.path(zPath), compactorId); + compactorLock = + new ServiceLock(getContext().getZooSession(), ServiceLock.path(zPath), compactorId); LockWatcher lw = new ServiceLockWatcher("compactor", () -> false, (name) -> getContext().getLowMemoryDetector().logGCInfo(getConfiguration())); diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java index 4e85fc0a7b0..f298e724d10 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java @@ -220,7 +220,7 @@ public Stream getReferences() { @Override public Map getTableIDs() throws InterruptedException { final String tablesPath = context.getZooKeeperRoot() + Constants.ZTABLES; - final ZooReader zr = context.getZooReader(); + final ZooReader zr = context.getZooSession().asReader(); int retries = 1; IllegalStateException ioe = null; while (retries <= 10) { diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java index ab5cc89df29..be415df136b 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java @@ -335,7 +335,7 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc var path = ServiceLock.path(getContext().getZooKeeperRoot() + Constants.ZGC_LOCK); UUID zooLockUUID = UUID.randomUUID(); - gcLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID); + gcLock = new ServiceLock(getContext().getZooSession(), path, zooLockUUID); HAServiceLockWatcher gcLockWatcher = new HAServiceLockWatcher("gc"); while (true) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index c52f6efcfb2..012c509a4f5 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -445,7 +445,7 @@ protected Manager(ConfigOpts opts, String[] args) throws IOException { log.info("SASL is enabled, creating delegation token key manager and distributor"); final long tokenUpdateInterval = aconf.getTimeInMillis(Property.GENERAL_DELEGATION_TOKEN_UPDATE_INTERVAL); - keyDistributor = new ZooAuthenticationKeyDistributor(context.getZooReaderWriter(), + keyDistributor = new ZooAuthenticationKeyDistributor(context.getZooSession(), context.getZooKeeperRoot() + Constants.ZDELEGATION_TOKEN_KEYS); authenticationTokenKeyManager = new AuthenticationTokenKeyManager(context.getSecretManager(), keyDistributor, tokenUpdateInterval, tokenLifetime); @@ -467,10 +467,10 @@ public MergeInfo getMergeInfo(TableId tableId) { synchronized (mergeLock) { try { String path = context.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge"; - if (!context.getZooReaderWriter().exists(path)) { + if (!context.getZooSession().asReaderWriter().exists(path)) { return new MergeInfo(); } - byte[] data = context.getZooReaderWriter().getData(path); + byte[] data = context.getZooSession().asReaderWriter().getData(path); DataInputBuffer in = new DataInputBuffer(); in.reset(data, data.length); MergeInfo info = new MergeInfo(); @@ -494,7 +494,7 @@ public void setMergeState(MergeInfo info, MergeState state) + info.getExtent().tableId() + "/merge"; info.setState(state); if (state.equals(MergeState.NONE)) { - context.getZooReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP); + context.getZooSession().asReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP); } else { DataOutputBuffer out = new DataOutputBuffer(); try { @@ -502,7 +502,7 @@ public void setMergeState(MergeInfo info, MergeState state) } catch (IOException ex) { throw new AssertionError("Unlikely", ex); } - context.getZooReaderWriter().putPersistentData(path, out.getData(), + context.getZooSession().asReaderWriter().putPersistentData(path, out.getData(), state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL : ZooUtil.NodeExistsPolicy.OVERWRITE); } @@ -514,7 +514,7 @@ public void setMergeState(MergeInfo info, MergeState state) public void clearMergeState(TableId tableId) throws KeeperException, InterruptedException { synchronized (mergeLock) { String path = getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge"; - getContext().getZooReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP); + getContext().getZooSession().asReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP); mergeLock.notifyAll(); } nextEvent.event("Merge state of %s cleared", tableId); @@ -522,7 +522,7 @@ public void clearMergeState(TableId tableId) throws KeeperException, Interrupted void setManagerGoalState(ManagerGoalState state) { try { - getContext().getZooReaderWriter().putPersistentData( + getContext().getZooSession().asReaderWriter().putPersistentData( getContext().getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, state.name().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE); } catch (Exception ex) { @@ -533,7 +533,7 @@ void setManagerGoalState(ManagerGoalState state) { ManagerGoalState getManagerGoalState() { while (true) { try { - byte[] data = getContext().getZooReaderWriter() + byte[] data = getContext().getZooSession().asReaderWriter() .getData(getContext().getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE); return ManagerGoalState.valueOf(new String(data, UTF_8)); } catch (Exception e) { @@ -750,7 +750,7 @@ private class ScanServerZKCleaner implements Runnable { @Override public void run() { - final ZooReaderWriter zrw = getContext().getZooReaderWriter(); + final ZooReaderWriter zrw = getContext().getZooSession().asReaderWriter(); final String sserverZNodePath = getContext().getZooKeeperRoot() + Constants.ZSSERVERS; while (stillManager()) { @@ -1293,7 +1293,7 @@ public void run() { Thread.currentThread().interrupt(); } - ZooReaderWriter zReaderWriter = context.getZooReaderWriter(); + ZooReaderWriter zReaderWriter = context.getZooSession().asReaderWriter(); try { zReaderWriter.getChildren(zroot + Constants.ZRECOVERY, new Watcher() { @@ -1363,7 +1363,7 @@ boolean canSuspendTablets() { final AgeOffStore store = new AgeOffStore<>( new org.apache.accumulo.core.fate.ZooStore<>( - context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()), + context.getZooKeeperRoot() + Constants.ZFATE, context.getZooSession()), HOURS.toMillis(8), System::currentTimeMillis); Fate f = initializeFateInstance(store, getConfiguration()); @@ -1564,7 +1564,7 @@ public ServiceLock getManagerLock() { private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc) throws KeeperException, InterruptedException { - var zooKeeper = getContext().getZooReaderWriter().getZooKeeper(); + var zooKeeper = getContext().getZooSession(); log.info("trying to get manager lock"); final String managerClientAddress = diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java index 02057c3b897..ca570bda1cf 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java @@ -109,7 +109,7 @@ public long initiateFlush(TInfo tinfo, TCredentials c, String tableIdStr) String zTablePath = manager.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_FLUSH_ID; - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); byte[] fid; try { fid = zoo.mutateExisting(zTablePath, currentValue -> { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java index b7cb2bfcdcb..807c9aed1b0 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java @@ -90,7 +90,7 @@ public class ManagerTime { public ManagerTime(Manager manager, AccumuloConfiguration conf) throws IOException { this.zPath = manager.getContext().getZooKeeperRoot() + Constants.ZMANAGER_TICK; - this.zk = manager.getContext().getZooReaderWriter(); + this.zk = manager.getContext().getZooSession().asReaderWriter(); this.manager = manager; try { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java index 702eaa7e565..ac6a0332dcf 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java @@ -146,7 +146,7 @@ public static FateMetricValues getFromZooKeeper(final ServerContext context, builder.withTxStateCounters(states); builder.withOpTypeCounters(opTypeCounters); - Stat node = context.getZooReaderWriter().getZooKeeper().exists(fateRootPath, false); + Stat node = context.getZooSession().exists(fateRootPath, null); builder.withZkFateChildOpsTotal(node.getCversion()); if (log.isTraceEnabled()) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java index 18e376e48ce..d0476081579 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java @@ -73,7 +73,7 @@ public FateMetrics(final ServerContext context, final long minimumRefreshDelay) this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, minimumRefreshDelay); try { - this.zooStore = new ZooStore<>(fateRootPath, context.getZooReaderWriter()); + this.zooStore = new ZooStore<>(fateRootPath, context.getZooSession()); } catch (KeeperException ex) { throw new IllegalStateException( "FATE Metrics - Failed to create zoo store - metrics unavailable", ex); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java index 7efd6f45118..f09fc8c1c3b 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java @@ -75,7 +75,7 @@ public RecoveryManager(Manager manager, long timeToCacheExistsInMillis) { executor = ThreadPools.getServerThreadPools().createScheduledExecutorService(4, "Walog sort starter"); - zooCache = new ZooCache(manager.getContext().getZooReader(), null); + zooCache = new ZooCache(manager.getContext().getZooSession()); try { List workIDs = new DistributedWorkQueue(manager.getContext().getZooKeeperRoot() + Constants.ZRECOVERY, diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java index c414cff441a..1f989ae7817 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java @@ -234,7 +234,7 @@ public static void main(String[] args) throws Exception { try (Scope scope = span.makeCurrent()) { try (AccumuloClient client = Accumulo.newClient().from(opts.getClientProps()).build()) { Map tableIdMap = client.tableOperations().tableIdMap(); - ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooReaderWriter(); + ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooSession().asReaderWriter(); for (Entry entry : tableIdMap.entrySet()) { final String table = entry.getKey(), tableId = entry.getValue(); String path = opts.getServerContext().getZooKeeperRoot() + Constants.ZTABLES + "/" 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 0914e675c1a..114b08af094 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 @@ -46,7 +46,7 @@ public static void main(String[] args) throws Exception { SecurityUtil.serverLogin(siteConfig); try (var context = new ServerContext(siteConfig)) { context.waitForZookeeperAndHdfs(); - context.getZooReaderWriter().putPersistentData( + context.getZooSession().asReaderWriter().putPersistentData( context.getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, args[0].getBytes(UTF_8), NodeExistsPolicy.OVERWRITE); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java index 8fc09eac2ca..ad9b75f1d9e 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java @@ -75,19 +75,20 @@ public static void checkTableNameDoesNotExist(ServerContext context, String tabl final Map namespaces = new HashMap<>(); final boolean namespaceInTableName = tableName.contains("."); try { - for (String tid : context.getZooReader() + for (String tid : context.getZooSession().asReader() .getChildren(context.getZooKeeperRoot() + Constants.ZTABLES)) { final String zTablePath = context.getZooKeeperRoot() + Constants.ZTABLES + "/" + tid; try { - final byte[] tname = context.getZooReader().getData(zTablePath + Constants.ZTABLE_NAME); + final byte[] tname = + context.getZooSession().asReader().getData(zTablePath + Constants.ZTABLE_NAME); Preconditions.checkState(tname != null, "Malformed table entry in ZooKeeper at %s", zTablePath); String namespaceName = Namespace.DEFAULT.name(); if (namespaceInTableName) { final byte[] nId = - context.getZooReader().getData(zTablePath + Constants.ZTABLE_NAMESPACE); + context.getZooSession().asReader().getData(zTablePath + Constants.ZTABLE_NAMESPACE); if (nId != null) { final NamespaceId namespaceId = NamespaceId.of(new String(nId, UTF_8)); if (!namespaceId.equals(Namespace.DEFAULT.id())) { @@ -128,7 +129,7 @@ public static void checkTableNameDoesNotExist(ServerContext context, String tabl public static > T getNextId(String name, ServerContext context, Function newIdFunction) throws AcceptableThriftTableOperationException { try { - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); final String ntp = context.getZooKeeperRoot() + Constants.ZTABLES; byte[] nid = zoo.mutateOrCreate(ntp, ZERO_BYTE, currentValue -> { BigInteger nextId = new BigInteger(new String(currentValue, UTF_8), Character.MAX_RADIX); @@ -150,7 +151,7 @@ public static long reserveTable(Manager env, TableId tableId, long tid, LockType boolean tableMustExist, TableOperation op) throws Exception { if (getLock(env.getContext(), tableId, tid, lockType).tryLock()) { if (tableMustExist) { - ZooReaderWriter zk = env.getContext().getZooReaderWriter(); + ZooReaderWriter zk = env.getContext().getZooSession().asReaderWriter(); if (!zk.exists(env.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId)) { throw new AcceptableThriftTableOperationException(tableId.canonical(), "", op, TableOperationExceptionType.NOTFOUND, "Table does not exist"); @@ -179,7 +180,7 @@ public static long reserveNamespace(Manager env, NamespaceId namespaceId, long i LockType lockType, boolean mustExist, TableOperation op) throws Exception { if (getLock(env.getContext(), namespaceId, id, lockType).tryLock()) { if (mustExist) { - ZooReaderWriter zk = env.getContext().getZooReaderWriter(); + ZooReaderWriter zk = env.getContext().getZooSession().asReaderWriter(); if (!zk.exists( env.getContext().getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId)) { throw new AcceptableThriftTableOperationException(namespaceId.canonical(), "", op, @@ -199,7 +200,7 @@ public static long reserveHdfsDirectory(Manager env, String directory, long tid) String resvPath = env.getContext().getZooKeeperRoot() + Constants.ZHDFS_RESERVATIONS + "/" + Base64.getEncoder().encodeToString(directory.getBytes(UTF_8)); - ZooReaderWriter zk = env.getContext().getZooReaderWriter(); + ZooReaderWriter zk = env.getContext().getZooSession().asReaderWriter(); if (ZooReservation.attempt(zk, resvPath, FastFormat.toHexString(tid), "")) { return 0; @@ -212,7 +213,7 @@ public static void unreserveHdfsDirectory(Manager env, String directory, long ti throws KeeperException, InterruptedException { String resvPath = env.getContext().getZooKeeperRoot() + Constants.ZHDFS_RESERVATIONS + "/" + Base64.getEncoder().encodeToString(directory.getBytes(UTF_8)); - ZooReservation.release(env.getContext().getZooReaderWriter(), resvPath, + ZooReservation.release(env.getContext().getZooSession().asReaderWriter(), resvPath, FastFormat.toHexString(tid)); } @@ -221,7 +222,7 @@ private static Lock getLock(ServerContext context, AbstractId id, long tid, byte[] lockData = FastFormat.toZeroPaddedHex(tid); var fLockPath = FateLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS + "/" + id.canonical()); - FateLock qlock = new FateLock(context.getZooReaderWriter(), fLockPath); + FateLock qlock = new FateLock(context.getZooSession().asReaderWriter(), fLockPath); DistributedLock lock = DistributedReadWriteLock.recoverLock(qlock, lockData); if (lock != null) { // Validate the recovered lock type diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java index 4c2fcb06457..61e97055300 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java @@ -100,7 +100,7 @@ public Repo call(final long tid, Manager env) throws Exception { String zTablePath = env.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_COMPACT_ID; - ZooReaderWriter zoo = env.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = env.getContext().getZooSession().asReaderWriter(); byte[] cid; try { cid = zoo.mutateExisting(zTablePath, currentValue -> { @@ -150,7 +150,7 @@ static void removeIterators(Manager environment, final long txid, TableId tableI String zTablePath = environment.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_COMPACT_ID; - ZooReaderWriter zoo = environment.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = environment.getContext().getZooSession().asReaderWriter(); try { zoo.mutateExisting(zTablePath, currentValue -> { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java index afcf127dce5..6625f9db757 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java @@ -88,7 +88,7 @@ public long isReady(long tid, Manager manager) throws Exception { String zCancelID = createCompactionCancellationPath(manager.getContext().getInstanceID(), tableId); - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); if (Long.parseLong(new String(zoo.getData(zCancelID), UTF_8)) >= compactId) { // compaction was canceled diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java index 0901edf9ead..d98e1dbbc53 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java @@ -73,7 +73,7 @@ public static void mutateZooKeeper(long tid, TableId tableId, Manager environmen String zCancelID = environment.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_COMPACT_CANCEL_ID; - ZooReaderWriter zoo = environment.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = environment.getContext().getZooSession().asReaderWriter(); byte[] currentValue = zoo.getData(zCompactID); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java index 77273d71d0d..cf9b2325f2f 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java @@ -61,7 +61,7 @@ private void preventFutureCompactions(Manager environment) throws KeeperException, InterruptedException { String deleteMarkerPath = createDeleteMarkerPath(environment.getContext().getInstanceID(), tableId); - ZooReaderWriter zoo = environment.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = environment.getContext().getZooSession().asReaderWriter(); zoo.putPersistentData(deleteMarkerPath, new byte[] {}, NodeExistsPolicy.SKIP); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java index 301f6e65e86..9383016a1a4 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java @@ -53,7 +53,7 @@ public Repo call(long tid, Manager manager) throws Exception { Utils.getTableNameLock().lock(); try { var context = manager.getContext(); - NamespaceMapping.put(context.getZooReaderWriter(), + NamespaceMapping.put(context.getZooSession().asReaderWriter(), context.getZooKeeperRoot() + Constants.ZNAMESPACES, namespaceInfo.namespaceId, namespaceInfo.namespaceName); TableManager.prepareNewNamespaceState(context, namespaceInfo.namespaceId, diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java index ebcb2e73ce9..4ceb5a33973 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java @@ -52,7 +52,7 @@ public RenameNamespace(NamespaceId namespaceId, String oldName, String newName) @Override public Repo call(long id, Manager manager) throws Exception { - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); Utils.getTableNameLock().lock(); try { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java index 4d605f6df01..54ad7df5926 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java @@ -73,7 +73,7 @@ public Repo call(long tid, Manager manager) throws Exception { "Namespace in new table name does not match the old table name"); } - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); Utils.getTableNameLock().lock(); try { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java index 4388ec65622..d127feeec24 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java @@ -95,7 +95,7 @@ public long isReady(long tid, Manager manager) { public Repo call(long tid, Manager manager) throws Exception { // suppress assignment of tablets to the server if (force) { - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); var zRoot = manager.getContext().getZooKeeperRoot(); var path = ServiceLock.path(zRoot + Constants.ZTSERVERS + "/" + hostAndPort); ServiceLock.deleteLock(zoo, path); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java index 2449355dad7..aaa6e8a6c16 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java @@ -23,14 +23,13 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.manager.EventCoordinator; import org.apache.accumulo.server.AccumuloDataVersion; import org.apache.accumulo.server.ServerContext; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; @@ -61,8 +60,7 @@ public void validate(final ServerContext context, final EventCoordinator eventCo private void validateACLs(ServerContext context) { final AtomicBoolean aclErrorOccurred = new AtomicBoolean(false); - final ZooReaderWriter zrw = context.getZooReaderWriter(); - final ZooKeeper zk = zrw.getZooKeeper(); + final ZooSession zk = context.getZooSession(); final String rootPath = context.getZooKeeperRoot(); final Set users = Set.of("accumulo", "anyone"); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java index f30b5b30bc2..6907e58bf7b 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java @@ -313,8 +313,8 @@ public UpgradeStatus getStatus() { justification = "Want to immediately stop all manager threads on upgrade error") private void abortIfFateTransactions(ServerContext context) { try { - final ReadOnlyTStore fate = new ZooStore<>( - context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()); + final ReadOnlyTStore fate = + new ZooStore<>(context.getZooKeeperRoot() + Constants.ZFATE, context.getZooSession()); if (!fate.list().isEmpty()) { throw new AccumuloException("Aborting upgrade because there are" + " outstanding FATE transactions from a previous Accumulo version." diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java index 4629f4842ac..236e229e341 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java @@ -79,7 +79,7 @@ public Upgrader10to11() { public void upgradeZookeeper(final ServerContext context) { log.info("upgrade of ZooKeeper entries"); - var zrw = context.getZooReaderWriter(); + var zrw = context.getZooSession().asReaderWriter(); var iid = context.getInstanceID(); // if the replication base path (../tables/+rep) assume removed or never existed. diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java index 704f3c3d509..a7a6eb07519 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java @@ -104,7 +104,7 @@ public void upgradeZookeeper(@NonNull ServerContext context) { var rootBase = zooRoot + ZROOT_TABLET; try { - var zrw = context.getZooReaderWriter(); + var zrw = context.getZooSession().asReaderWriter(); // clean up nodes no longer in use zrw.recursiveDelete(zooRoot + ZTRACERS, ZooUtil.NodeMissingPolicy.SKIP); @@ -308,16 +308,17 @@ public void createScanServerRefTable(ServerContext context) { private void removeZKProblemReports(ServerContext context) { String zpath = context.getZooKeeperRoot() + ZPROBLEMS; try { - if (!context.getZooReaderWriter().exists(zpath)) { + if (!context.getZooSession().asReaderWriter().exists(zpath)) { // could be running a second time and the node was already deleted return; } - var children = context.getZooReaderWriter().getChildren(zpath); + var children = context.getZooSession().asReaderWriter().getChildren(zpath); for (var child : children) { var pr = ProblemReport.decodeZooKeeperEntry(context, child); logProblemDeletion(pr); } - context.getZooReaderWriter().recursiveDelete(zpath, ZooUtil.NodeMissingPolicy.SKIP); + context.getZooSession().asReaderWriter().recursiveDelete(zpath, + ZooUtil.NodeMissingPolicy.SKIP); } catch (Exception e) { throw new IllegalStateException(e); } @@ -421,7 +422,7 @@ static ProblemReport decodeZooKeeperEntry(ServerContext context, String node) String resource = dis.readUTF(); String zpath = context.getZooKeeperRoot() + ZPROBLEMS + "/" + node; - byte[] enc = context.getZooReaderWriter().getData(zpath); + byte[] enc = context.getZooSession().asReaderWriter().getData(zpath); return new ProblemReport(tableId, ProblemType.valueOf(problemType), resource, enc); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriverTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriverTest.java index 8cfd2d0c63b..93eb624a5af 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriverTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriverTest.java @@ -36,90 +36,70 @@ import org.apache.accumulo.core.data.NamespaceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.delete.PreDeleteTable; import org.apache.accumulo.server.ServerContext; +import org.apache.zookeeper.data.Stat; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; public class CompactionDriverTest { - @Test - public void testCancelId() throws Exception { - - final InstanceId instance = InstanceId.of(UUID.randomUUID()); - final long compactId = 123; - final long cancelId = 124; - final NamespaceId namespaceId = NamespaceId.of("13"); - final TableId tableId = TableId.of("42"); - final byte[] startRow = new byte[0]; - final byte[] endRow = new byte[0]; - - Manager manager = createMock(Manager.class); - ServerContext ctx = createMock(ServerContext.class); - ZooReaderWriter zrw = createMock(ZooReaderWriter.class); + private final InstanceId instance = InstanceId.of(UUID.randomUUID()); + private final NamespaceId namespaceId = NamespaceId.of("testNamespace"); + private final TableId tableId = TableId.of("testTable"); + private final byte[] startRow = new byte[0]; + private final byte[] endRow = new byte[0]; + private final long compactId = 123; // arbitrary + + private Manager manager; + private ServerContext ctx; + private ZooSession zk; + + @BeforeEach + public void setup() { + manager = createMock(Manager.class); + ctx = createMock(ServerContext.class); + zk = createMock(ZooSession.class); expect(ctx.getInstanceID()).andReturn(instance).anyTimes(); - expect(ctx.getZooReaderWriter()).andReturn(zrw).anyTimes(); + expect(ctx.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReaderWriter()).andReturn(new ZooReaderWriter(zk)).anyTimes(); expect(manager.getContext()).andReturn(ctx).anyTimes(); + } - final String zCancelID = CompactionDriver.createCompactionCancellationPath(instance, tableId); - expect(zrw.getData(zCancelID)).andReturn(Long.toString(cancelId).getBytes(UTF_8)); - - replay(manager, ctx, zrw); - - final CompactionDriver driver = - new CompactionDriver(compactId, namespaceId, tableId, startRow, endRow); - final long tableIdLong = Long.parseLong(tableId.toString()); - - var e = assertThrows(AcceptableThriftTableOperationException.class, - () -> driver.isReady(tableIdLong, manager)); - - assertEquals(e.getTableId(), tableId.toString()); - assertEquals(e.getOp(), TableOperation.COMPACT); - assertEquals(e.getType(), TableOperationExceptionType.OTHER); - assertEquals(TableOperationsImpl.COMPACTION_CANCELED_MSG, e.getDescription()); + @AfterEach + public void teardown() { + verify(manager, ctx, zk); + } - verify(manager, ctx, zrw); + @Test + public void testCancelId() throws Exception { + runDriver(compactId + 1, TableOperationsImpl.COMPACTION_CANCELED_MSG); } @Test public void testTableBeingDeleted() throws Exception { - - final InstanceId instance = InstanceId.of(UUID.randomUUID()); - final long compactId = 123; - final long cancelId = 122; - final NamespaceId namespaceId = NamespaceId.of("14"); - final TableId tableId = TableId.of("43"); - final byte[] startRow = new byte[0]; - final byte[] endRow = new byte[0]; - - Manager manager = createMock(Manager.class); - ServerContext ctx = createMock(ServerContext.class); - ZooReaderWriter zrw = createMock(ZooReaderWriter.class); - expect(ctx.getInstanceID()).andReturn(instance).anyTimes(); - expect(ctx.getZooReaderWriter()).andReturn(zrw).anyTimes(); - expect(manager.getContext()).andReturn(ctx).anyTimes(); - - final String zCancelID = CompactionDriver.createCompactionCancellationPath(instance, tableId); - expect(zrw.getData(zCancelID)).andReturn(Long.toString(cancelId).getBytes(UTF_8)); - String deleteMarkerPath = PreDeleteTable.createDeleteMarkerPath(instance, tableId); - expect(zrw.exists(deleteMarkerPath)).andReturn(true); + expect(zk.exists(deleteMarkerPath, null)).andReturn(new Stat()).once(); + runDriver(compactId - 1, TableOperationsImpl.TABLE_DELETED_MSG); + } - replay(manager, ctx, zrw); + private void runDriver(long cancelId, String expectedMessage) throws Exception { + final String zCancelID = CompactionDriver.createCompactionCancellationPath(instance, tableId); + expect(zk.getData(zCancelID, null, null)).andReturn(Long.toString(cancelId).getBytes(UTF_8)); - final CompactionDriver driver = - new CompactionDriver(compactId, namespaceId, tableId, startRow, endRow); - final long tableIdLong = Long.parseLong(tableId.toString()); + final var driver = new CompactionDriver(compactId, namespaceId, tableId, startRow, endRow); + final long mockTxId = tableId.hashCode(); + replay(manager, ctx, zk); var e = assertThrows(AcceptableThriftTableOperationException.class, - () -> driver.isReady(tableIdLong, manager)); - + () -> driver.isReady(mockTxId, manager)); assertEquals(e.getTableId(), tableId.toString()); assertEquals(e.getOp(), TableOperation.COMPACT); assertEquals(e.getType(), TableOperationExceptionType.OTHER); - assertEquals(TableOperationsImpl.TABLE_DELETED_MSG, e.getDescription()); - - verify(manager, ctx, zrw); + assertEquals(expectedMessage, e.getDescription()); } } diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java index 0b756f000bb..cbfb5dd32a3 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java @@ -38,14 +38,16 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.core.metadata.AccumuloTable; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.PropStore; import org.apache.accumulo.server.conf.store.TablePropKey; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -54,101 +56,96 @@ class Upgrader10to11Test { private static final Logger log = LoggerFactory.getLogger(Upgrader10to11Test.class); - private InstanceId instanceId = null; - private ServerContext context = null; - private ZooReaderWriter zrw = null; - - private PropStore propStore = null; + private InstanceId instanceId; + private ServerContext context; + private ZooSession zk; + private PropStore propStore; @BeforeEach public void initMocks() { instanceId = InstanceId.of(UUID.randomUUID()); context = createMock(ServerContext.class); - zrw = createMock(ZooReaderWriter.class); + zk = createMock(ZooSession.class); propStore = createMock(PropStore.class); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); + expect(context.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReaderWriter()).andReturn(new ZooReaderWriter(zk)).anyTimes(); expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); } + @AfterEach + public void verifyMocks() { + verify(context, zk, propStore); + } + @Test void upgradeZooKeeperGoPath() throws Exception { - expect(context.getPropStore()).andReturn(propStore).anyTimes(); - expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once(); - expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE)) + expect(zk.exists(buildRepTablePath(instanceId), null)).andReturn(new Stat()).once(); + expect(zk.getData(buildRepTablePath(instanceId) + ZTABLE_STATE, null, null)) .andReturn(TableState.OFFLINE.name().getBytes(UTF_8)).once(); - zrw.recursiveDelete(buildRepTablePath(instanceId), ZooUtil.NodeMissingPolicy.SKIP); + expect(zk.getChildren(buildRepTablePath(instanceId), null)).andReturn(List.of()); + zk.delete(buildRepTablePath(instanceId), -1); expectLastCall().once(); expect(propStore.get(TablePropKey.of(instanceId, AccumuloTable.METADATA.tableId()))) .andReturn(new VersionedProperties()).once(); - replay(context, zrw, propStore); + replay(context, zk, propStore); Upgrader10to11 upgrader = new Upgrader10to11(); upgrader.upgradeZookeeper(context); - - verify(context, zrw); } @Test void upgradeZookeeperNoReplTableNode() throws Exception { - - expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(false).once(); - replay(context, zrw); + expect(zk.exists(buildRepTablePath(instanceId), null)).andReturn(null).once(); + replay(context, zk, propStore); Upgrader10to11 upgrader = new Upgrader10to11(); upgrader.upgradeZookeeper(context); - - verify(context, zrw); } @Test void checkReplicationStateOffline() throws Exception { expect(context.getPropStore()).andReturn(propStore).anyTimes(); - expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once(); - expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE)) + expect(zk.exists(buildRepTablePath(instanceId), null)).andReturn(new Stat()).once(); + expect(zk.getData(buildRepTablePath(instanceId) + ZTABLE_STATE, null, null)) .andReturn(TableState.OFFLINE.name().getBytes(UTF_8)).once(); - zrw.recursiveDelete(buildRepTablePath(instanceId), ZooUtil.NodeMissingPolicy.SKIP); + expect(zk.getChildren(buildRepTablePath(instanceId), null)).andReturn(List.of()); + zk.delete(buildRepTablePath(instanceId), -1); expectLastCall().once(); expect(propStore.get(TablePropKey.of(instanceId, AccumuloTable.METADATA.tableId()))) .andReturn(new VersionedProperties()).once(); - replay(context, zrw, propStore); + replay(context, zk, propStore); Upgrader10to11 upgrader = new Upgrader10to11(); upgrader.upgradeZookeeper(context); - - verify(context, zrw); } @Test void checkReplicationStateOnline() throws Exception { - expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once(); - expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE)) + expect(zk.exists(buildRepTablePath(instanceId), null)).andReturn(new Stat()).once(); + expect(zk.getData(buildRepTablePath(instanceId) + ZTABLE_STATE, null, null)) .andReturn(TableState.ONLINE.name().getBytes(UTF_8)).anyTimes(); - replay(context, zrw); + replay(context, zk, propStore); Upgrader10to11 upgrader = new Upgrader10to11(); assertThrows(IllegalStateException.class, () -> upgrader.upgradeZookeeper(context)); - - verify(context, zrw); } @Test void checkReplicationStateNoNode() throws Exception { - expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once(); - expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE)) + expect(zk.exists(buildRepTablePath(instanceId), null)).andReturn(new Stat()).once(); + expect(zk.getData(buildRepTablePath(instanceId) + ZTABLE_STATE, null, null)) .andThrow(new KeeperException.NoNodeException("force no node exception")).anyTimes(); - replay(context, zrw); + replay(context, zk, propStore); Upgrader10to11 upgrader = new Upgrader10to11(); assertThrows(IllegalStateException.class, () -> upgrader.upgradeZookeeper(context)); - - verify(context, zrw); } @Test @@ -188,5 +185,7 @@ public void filterTest() { assertEquals(6, filtered.size()); log.info("F:{}", filtered); + + replay(context, zk, propStore); } } diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java index ca5e57e7a3a..c3a564d6f80 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java @@ -43,7 +43,6 @@ import java.util.Map; import java.util.TreeMap; import java.util.UUID; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.accumulo.core.Constants; @@ -63,6 +62,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily; import org.apache.accumulo.core.metadata.schema.RootTabletMetadata; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; @@ -352,11 +352,13 @@ public void upgradeZooKeeperTest() throws Exception { Upgrader11to12 upgrader = new Upgrader11to12(); ServerContext context = createMock(ServerContext.class); + ZooSession zk = createStrictMock(ZooSession.class); ZooReaderWriter zrw = createStrictMock(ZooReaderWriter.class); final var zkRoot = ZooUtil.getRoot(iid); expect(context.getInstanceID()).andReturn(iid).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); + expect(context.getZooSession()).andReturn(zk).anyTimes(); + expect(zk.asReaderWriter()).andReturn(zrw).anyTimes(); expect(context.getZooKeeperRoot()).andReturn(zkRoot).anyTimes(); zrw.recursiveDelete(zkRoot + "/tracers", ZooUtil.NodeMissingPolicy.SKIP); @@ -382,30 +384,26 @@ public void upgradeZooKeeperTest() throws Exception { expect(zrw.getChildren(eq(zkRoot + Constants.ZNAMESPACES))) .andReturn(List.copyOf(mockNamespaces.keySet())).once(); for (String ns : mockNamespaces.keySet()) { - Supplier pathMatcher = - () -> eq(zkRoot + Constants.ZNAMESPACES + "/" + ns + ZNAMESPACE_NAME); - expect(zrw.getData(pathMatcher.get())).andReturn(mockNamespaces.get(ns).getBytes(UTF_8)) - .once(); + expect(zrw.getData(zkRoot + Constants.ZNAMESPACES + "/" + ns + ZNAMESPACE_NAME)) + .andReturn(mockNamespaces.get(ns).getBytes(UTF_8)).once(); } byte[] mapping = NamespaceMapping.serialize(mockNamespaces); expect(zrw.putPersistentData(eq(zkRoot + Constants.ZNAMESPACES), aryEq(mapping), eq(ZooUtil.NodeExistsPolicy.OVERWRITE))).andReturn(true).once(); for (String ns : mockNamespaces.keySet()) { - Supplier pathMatcher = - () -> eq(zkRoot + Constants.ZNAMESPACES + "/" + ns + ZNAMESPACE_NAME); - zrw.delete(pathMatcher.get()); + zrw.delete(zkRoot + Constants.ZNAMESPACES + "/" + ns + ZNAMESPACE_NAME); expectLastCall().once(); } - expect(zrw.exists(eq(zkRoot + "/problems"))).andReturn(false).once(); + expect(zrw.exists(zkRoot + "/problems")).andReturn(false).once(); - replay(context, zrw); + replay(context, zk, zrw); upgrader.upgradeZookeeper(context); assertEquals(zKRootV2, new String(byteCapture.getValue(), UTF_8)); - verify(context, zrw); + verify(context, zk, zrw); } @Test diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index 839dfd0eaef..cb09d740956 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -324,7 +324,7 @@ private GCStatus fetchGcStatus() { HostAndPort address = null; try { // Read the gc location from its lock - ZooReaderWriter zk = context.getZooReaderWriter(); + ZooReaderWriter zk = context.getZooSession().asReaderWriter(); var path = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZGC_LOCK); List locks = ServiceLock.validateAndSort(path, zk.getChildren(path.toString())); if (locks != null && !locks.isEmpty()) { @@ -399,7 +399,7 @@ public void run() { try { URL url = new URL(server.isSecure() ? "https" : "http", advertiseHost, server.getPort(), "/"); final String path = context.getZooKeeperRoot() + Constants.ZMONITOR_HTTP_ADDR; - final ZooReaderWriter zoo = context.getZooReaderWriter(); + final ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); // Delete before we try to re-create in case the previous session hasn't yet expired zoo.delete(path); zoo.putEphemeralData(path, url.toString().getBytes(UTF_8)); @@ -723,7 +723,7 @@ private void getMonitorLock(HostAndPort monitorLocation) final var monitorLockPath = ServiceLock.path(zRoot + Constants.ZMONITOR_LOCK); // Ensure that everything is kosher with ZK as this has changed. - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); if (zoo.exists(monitorPath)) { byte[] data = zoo.getData(monitorPath); // If the node isn't empty, it's from a previous install (has hostname:port for HTTP server) @@ -750,7 +750,7 @@ private void getMonitorLock(HostAndPort monitorLocation) // Get a ZooLock for the monitor UUID zooLockUUID = UUID.randomUUID(); - monitorLock = new ServiceLock(zoo.getZooKeeper(), monitorLockPath, zooLockUUID); + monitorLock = new ServiceLock(context.getZooSession(), monitorLockPath, zooLockUUID); while (true) { HAServiceLockWatcher monitorLockWatcher = new HAServiceLockWatcher("monitor"); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java index 8db73cb001a..1d76c2596b9 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java @@ -218,7 +218,7 @@ public ScanServer(ConfigOpts opts, String[] args) { this.resourceManager = new TabletServerResourceManager(context, this); - this.managerLockCache = new ZooCache(context.getZooReader(), null); + this.managerLockCache = new ZooCache(context.getZooSession()); var readWriteLock = new ReentrantReadWriteLock(); reservationsReadLock = readWriteLock.readLock(); @@ -326,7 +326,7 @@ public String getClientAddressString() { * Set up nodes and locks in ZooKeeper for this Compactor */ private ServiceLock announceExistence() { - ZooReaderWriter zoo = getContext().getZooReaderWriter(); + ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { var zLockPath = ServiceLock.path( @@ -344,7 +344,7 @@ private ServiceLock announceExistence() { } serverLockUUID = UUID.randomUUID(); - scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID); + scanServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, serverLockUUID); LockWatcher lw = new ServiceLockWatcher("scan server", () -> serverStopRequested, (name) -> context.getLowMemoryDetector().logGCInfo(getConfiguration())); 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 40d8886cfb0..53af48d7338 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 @@ -244,7 +244,7 @@ public static void main(String[] args) throws Exception { protected TabletServer(ConfigOpts opts, String[] args) { super("tserver", opts, args); context = super.getContext(); - this.managerLockCache = new ZooCache(context.getZooReader(), null); + this.managerLockCache = new ZooCache(context.getZooSession()); final AccumuloConfiguration aconf = getConfiguration(); log.info("Version " + Constants.VERSION); log.info("Instance " + getInstanceID()); @@ -346,9 +346,8 @@ private void logBusyTablets(List> busyTablets, if (aconf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) { log.info("SASL is enabled, creating ZooKeeper watcher for AuthenticationKeys"); // Watcher to notice new AuthenticationKeys which enable delegation tokens - authKeyWatcher = - new ZooAuthenticationKeyWatcher(context.getSecretManager(), context.getZooReaderWriter(), - context.getZooKeeperRoot() + Constants.ZDELEGATION_TOKEN_KEYS); + authKeyWatcher = new ZooAuthenticationKeyWatcher(context.getSecretManager(), + context.getZooSession(), context.getZooKeeperRoot() + Constants.ZDELEGATION_TOKEN_KEYS); } else { authKeyWatcher = null; } @@ -616,7 +615,7 @@ public ZooCache getManagerLockCache() { } private void announceExistence() { - ZooReaderWriter zoo = getContext().getZooReaderWriter(); + ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { var zLockPath = ServiceLock.path( getContext().getZooKeeperRoot() + Constants.ZTSERVERS + "/" + getClientAddressString()); @@ -632,7 +631,7 @@ private void announceExistence() { } UUID tabletServerUUID = UUID.randomUUID(); - tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, tabletServerUUID); + tabletServerLock = new ServiceLock(getContext().getZooSession(), zLockPath, tabletServerUUID); LockWatcher lw = new ServiceLockWatcher("tablet server", () -> serverStopRequested, (name) -> context.getLowMemoryDetector().logGCInfo(getConfiguration())); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java index 675168043ce..b797c6c472c 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java @@ -678,7 +678,7 @@ public long getFlushID() throws NoNodeException { try { String zTablePath = tabletServer.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + extent.tableId() + Constants.ZTABLE_FLUSH_ID; - String id = new String(context.getZooReaderWriter().getData(zTablePath), UTF_8); + String id = new String(context.getZooSession().asReaderWriter().getData(zTablePath), UTF_8); return Long.parseLong(id); } catch (InterruptedException | NumberFormatException e) { throw new RuntimeException("Exception on " + extent + " getting flush ID", e); @@ -704,7 +704,8 @@ public Pair getCompactionID() throws NoNodeException { + extent.tableId() + Constants.ZTABLE_COMPACT_ID; String[] tokens = - new String(context.getZooReaderWriter().getData(zTablePath), UTF_8).split(","); + new String(context.getZooSession().asReaderWriter().getData(zTablePath), UTF_8) + .split(","); long compactID = Long.parseLong(tokens[0]); CompactionConfig overlappingConfig = null; diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java index 289f71d5049..1ade1cb2b61 100644 --- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java @@ -115,7 +115,7 @@ public void testExistingInstance() throws Exception { } } - ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getCluster().getServerContext().getZooSession().asReaderWriter(); final String zInstanceRoot = getCluster().getServerContext().getZooKeeperRoot(); while (!AccumuloStatus.isAccumuloOffline(zrw, zInstanceRoot)) { log.debug("Accumulo services still have their ZK locks held"); diff --git a/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java index 7830de57d9e..3eed1bab868 100644 --- a/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/FateSummaryIT.java @@ -50,9 +50,9 @@ import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.ZooStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.util.FastFormat; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.minicluster.ServerType; import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; @@ -199,10 +199,10 @@ public void testFatePrintAndSummaryCommandsWithInProgressTxns() throws Exception // This error was occurring in AdminUtil.getTransactionStatus(). One of the methods that is // called which may throw the NNE is top(), so we will mock this method to sometimes throw a // NNE and ensure it is handled/ignored within getTransactionStatus() - ZooStore zs = EasyMock.createMockBuilder(ZooStore.class) - .withConstructor(String.class, ZooReaderWriter.class) - .withArgs(sctx.getZooKeeperRoot() + Constants.ZFATE, sctx.getZooReaderWriter()) - .addMockedMethod("top").addMockedMethod("list").createMock(); + ZooStore zs = + EasyMock.createMockBuilder(ZooStore.class).withConstructor(String.class, ZooSession.class) + .withArgs(sctx.getZooKeeperRoot() + Constants.ZFATE, sctx.getZooSession()) + .addMockedMethod("top").addMockedMethod("list").createMock(); // Create 3 transactions, when iterating through the list of transactions in // getTransactionStatus(), the 2nd transaction should cause a NNE which should be // handled/ignored in getTransactionStatus(). The other two transactions should still diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java index 8e22b9e2031..ffc4d59b82c 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java @@ -86,7 +86,7 @@ private void startScanServer(String cacheExpiration, String cacheRefresh) throws IOException, KeeperException, InterruptedException { String zooRoot = getCluster().getServerContext().getZooKeeperRoot(); - ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getCluster().getServerContext().getZooSession().asReaderWriter(); String scanServerRoot = zooRoot + Constants.ZSSERVERS; SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerGroupConfigurationIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerGroupConfigurationIT.java index 0e2aa02e8d5..5d67ce6df00 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerGroupConfigurationIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerGroupConfigurationIT.java @@ -31,6 +31,7 @@ import org.apache.accumulo.core.conf.ClientProperty; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.spi.scan.ScanServerSelector; import org.apache.accumulo.harness.MiniClusterConfigurationCallback; @@ -40,7 +41,6 @@ import org.apache.accumulo.test.util.Wait; import org.apache.accumulo.tserver.ScanServer; import org.apache.hadoop.conf.Configuration; -import org.apache.zookeeper.ZooKeeper; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -127,11 +127,11 @@ public static void after() throws Exception { public void testClientConfiguration() throws Exception { final String zooRoot = getCluster().getServerContext().getZooKeeperRoot(); - final ZooKeeper zk = getCluster().getServerContext().getZooReaderWriter().getZooKeeper(); + final ZooReaderWriter zk = getCluster().getServerContext().getZooSession().asReaderWriter(); final String scanServerRoot = zooRoot + Constants.ZSSERVERS; // Ensure no scan servers running - Wait.waitFor(() -> zk.getChildren(scanServerRoot, false).size() == 0); + Wait.waitFor(() -> zk.getChildren(scanServerRoot).size() == 0); try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { final String tableName = getUniqueNames(1)[0]; @@ -152,7 +152,7 @@ public void testClientConfiguration() throws Exception { // Start a ScanServer. No group specified, should be in the default group. getCluster().getClusterControl().start(ServerType.SCAN_SERVER, "localhost"); - Wait.waitFor(() -> zk.getChildren(scanServerRoot, false).size() == 1, 30_000); + Wait.waitFor(() -> zk.getChildren(scanServerRoot).size() == 1, 30_000); Wait.waitFor(() -> ((ClientContext) client).getScanServers().values().stream().anyMatch( (p) -> p.getSecond().equals(ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME)) == true); @@ -172,7 +172,7 @@ public void testClientConfiguration() throws Exception { // and the scripts need to be updated. getCluster()._exec(ScanServer.class, ServerType.SCAN_SERVER, Map.of(), new String[] {"-o", "sserver.group=GROUP1"}); - Wait.waitFor(() -> zk.getChildren(scanServerRoot, false).size() == 2); + Wait.waitFor(() -> zk.getChildren(scanServerRoot).size() == 2); Wait.waitFor(() -> ((ClientContext) client).getScanServers().values().stream().anyMatch( (p) -> p.getSecond().equals(ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME)) == true); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java index 77cdc8c6478..5daa1425021 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java @@ -84,7 +84,7 @@ public static void start() throws Exception { "localhost"); String zooRoot = getCluster().getServerContext().getZooKeeperRoot(); - ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getCluster().getServerContext().getZooSession().asReaderWriter(); String scanServerRoot = zooRoot + Constants.ZSSERVERS; while (zrw.getChildren(scanServerRoot).size() == 0) { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java index 9aef8ac676e..3c1239028bf 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java @@ -92,7 +92,7 @@ public static void start() throws Exception { "localhost"); String zooRoot = getCluster().getServerContext().getZooKeeperRoot(); - ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getCluster().getServerContext().getZooSession().asReaderWriter(); String scanServerRoot = zooRoot + Constants.ZSSERVERS; while (zrw.getChildren(scanServerRoot).size() == 0) { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java index 6283037c801..a55a55278ad 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java @@ -90,7 +90,7 @@ public static void start() throws Exception { "localhost"); String zooRoot = getCluster().getServerContext().getZooKeeperRoot(); - ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getCluster().getServerContext().getZooSession().asReaderWriter(); String scanServerRoot = zooRoot + Constants.ZSSERVERS; while (zrw.getChildren(scanServerRoot).size() == 0) { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerShutdownIT.java index 5633474aaea..7e165c5921d 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerShutdownIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerShutdownIT.java @@ -84,7 +84,7 @@ public void testRefRemovalOnShutdown() throws Exception { ServerContext ctx = getCluster().getServerContext(); String zooRoot = ctx.getZooKeeperRoot(); - ZooReaderWriter zrw = ctx.getZooReaderWriter(); + ZooReaderWriter zrw = ctx.getZooSession().asReaderWriter(); String scanServerRoot = zooRoot + Constants.ZSSERVERS; Wait.waitFor(() -> zrw.getChildren(scanServerRoot).size() == 0); diff --git a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java index 41a3f76e639..14a86bd20e5 100644 --- a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java @@ -138,7 +138,7 @@ public void testManagerService() throws Exception { // Wait for the Manager to grab its lock while (true) { try { - List locks = cluster.getServerContext().getZooReader() + List locks = cluster.getServerContext().getZooSession().asReader() .getChildren(cluster.getServerContext().getZooKeeperRoot() + Constants.ZMANAGER_LOCK); if (!locks.isEmpty()) { break; @@ -196,7 +196,7 @@ public void testGarbageCollectorPorts() throws Exception { // Wait for the Manager to grab its lock while (true) { try { - List locks = cluster.getServerContext().getZooReader() + List locks = cluster.getServerContext().getZooSession().asReader() .getChildren(cluster.getServerContext().getZooKeeperRoot() + Constants.ZGC_LOCK); if (!locks.isEmpty()) { break; diff --git a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java index 0fb40ab9dea..2edf7eb0879 100644 --- a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java +++ b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java @@ -243,7 +243,7 @@ public void permissionsTest() throws Exception { Thread.sleep(SECONDS.toMillis(3L)); ServerContext serverContext = getCluster().getServerContext(); - ZooReaderWriter zrw = serverContext.getZooReaderWriter(); + ZooReaderWriter zrw = serverContext.getZooSession().asReaderWriter(); // validate that a world-readable node has expected perms to validate test method var noAcl = zrw.getACL(ZooUtil.getRoot(serverContext.getInstanceID())); diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java index bb96d3c7471..1bbb80ba31c 100644 --- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java +++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java @@ -24,10 +24,6 @@ import static org.apache.accumulo.core.conf.Property.TSERV_NATIVEMAP_ENABLED; import static org.apache.accumulo.core.conf.Property.TSERV_SCAN_MAX_OPENFILES; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -41,7 +37,8 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; import org.apache.accumulo.server.conf.store.TablePropKey; @@ -51,9 +48,7 @@ import org.apache.accumulo.server.conf.store.impl.ZooPropLoader; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -72,34 +67,25 @@ public class PropCacheCaffeineImplZkIT { private static ZooKeeperTestingServer testZk = null; private static ZooReaderWriter zrw; - private static ZooKeeper zooKeeper; + private static ZooSession zk; private final TableId tIdA = TableId.of("A"); private final TableId tIdB = TableId.of("B"); - private static ServerContext context; @TempDir private static File tempDir; @BeforeAll public static void setupZk() throws Exception { - // using default zookeeper port - we don't have a full configuration testZk = new ZooKeeperTestingServer(tempDir); - zooKeeper = testZk.newClient(); - - zrw = testZk.getZooReaderWriter(); - context = createNiceMock(ServerContext.class); - expect(context.getInstanceID()).andReturn(INSTANCE_ID).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); - - replay(context); + zk = testZk.newClient(); + zrw = zk.asReaderWriter(); } @AfterAll public static void shutdownZK() throws Exception { - verify(context); try { - zooKeeper.close(); + zk.close(); } finally { testZk.close(); } @@ -108,24 +94,22 @@ public static void shutdownZK() throws Exception { @BeforeEach public void setupZnodes() throws Exception { zrw.mkdirs(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZCONFIG); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES, new byte[0], + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical(), + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create( - ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical() + "/conf", + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdA.canonical() + "/conf", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical(), + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create( - ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical() + "/conf", + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tIdB.canonical() + "/conf", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } @AfterEach public void cleanupZnodes() throws Exception { - ZKUtil.deleteRecursive(zooKeeper, "/accumulo"); + ZKUtil.deleteRecursive(zk, "/accumulo"); } @Test @@ -145,12 +129,11 @@ public void init() throws Exception { assertTrue(created, "expected properties to be created"); - ReadyMonitor readyMonitor = new ReadyMonitor("test", zooKeeper.getSessionTimeout()); + ReadyMonitor readyMonitor = new ReadyMonitor("test", zk.getSessionTimeout()); PropStoreWatcher propStoreWatcher = new PropStoreWatcher(readyMonitor); - ZooPropLoader propLoader = - new ZooPropLoader(zrw, VersionedPropCodec.getDefault(), propStoreWatcher); + var propLoader = new ZooPropLoader(zk, VersionedPropCodec.getDefault(), propStoreWatcher); PropCacheCaffeineImpl cache = new PropCacheCaffeineImpl.Builder(propLoader).build(); VersionedProperties readProps = cache.get(propStoreKey); diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java index f185f933e09..d74dac8cc79 100644 --- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java +++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java @@ -19,6 +19,7 @@ package org.apache.accumulo.test.conf.store; import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER; +import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -40,6 +41,8 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.codec.VersionedPropCodec; import org.apache.accumulo.server.conf.codec.VersionedProperties; @@ -51,11 +54,8 @@ import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; -import org.easymock.EasyMock; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -72,7 +72,7 @@ public class PropStoreZooKeeperIT { private static final Logger log = LoggerFactory.getLogger(PropStoreZooKeeperIT.class); private static final VersionedPropCodec propCodec = VersionedPropCodec.getDefault(); private static ZooKeeperTestingServer testZk = null; - private static ZooKeeper zooKeeper; + private static ZooSession zk; private ServerContext context; private InstanceId instanceId = null; private PropStore propStore = null; @@ -84,16 +84,14 @@ public class PropStoreZooKeeperIT { @BeforeAll public static void setupZk() throws Exception { - // using default zookeeper port - we don't have a full configuration testZk = new ZooKeeperTestingServer(tempDir); - zooKeeper = testZk.newClient(); - ZooUtil.digestAuth(zooKeeper, ZooKeeperTestingServer.SECRET); + zk = testZk.newClient(); } @AfterAll public static void shutdownZK() throws Exception { try { - zooKeeper.close(); + zk.close(); } finally { testZk.close(); } @@ -101,34 +99,31 @@ public static void shutdownZK() throws Exception { @BeforeEach public void setupZnodes() throws Exception { - var zrw = testZk.getZooReaderWriter(); instanceId = InstanceId.of(UUID.randomUUID()); - context = EasyMock.createNiceMock(ServerContext.class); + context = createMock(ServerContext.class); expect(context.getInstanceID()).andReturn(instanceId).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); + expect(context.getZooSession()).andReturn(zk).anyTimes(); replay(context); - zrw.mkdirs(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG); - zooKeeper.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES, new byte[0], + zk.asReaderWriter().mkdirs(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG); + zk.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdA.canonical(), - new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create( - ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdA.canonical() + "/conf", + zk.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdA.canonical(), new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdA.canonical() + "/conf", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdB.canonical(), - new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create( - ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdB.canonical() + "/conf", + zk.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdB.canonical(), new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create(ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tIdB.canonical() + "/conf", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - propStore = ZooPropStore.initialize(instanceId, context.getZooReaderWriter()); + propStore = ZooPropStore.initialize(instanceId, context.getZooSession()); } @AfterEach public void cleanupZnodes() throws Exception { - ZKUtil.deleteRecursive(zooKeeper, "/accumulo"); + ZKUtil.deleteRecursive(zk, "/accumulo"); } /** @@ -139,7 +134,7 @@ public void createNoProps() throws InterruptedException, KeeperException { var propKey = TablePropKey.of(instanceId, tIdA); // read from ZK, after delete no node and node not created. - assertNull(zooKeeper.exists(propKey.getPath(), false)); + assertNull(zk.exists(propKey.getPath(), null)); assertThrows(IllegalStateException.class, () -> propStore.get(propKey)); } @@ -147,12 +142,12 @@ public void createNoProps() throws InterruptedException, KeeperException { public void failOnDuplicate() throws InterruptedException, KeeperException { var propKey = TablePropKey.of(instanceId, tIdA); - assertNull(zooKeeper.exists(propKey.getPath(), false)); // check node does not exist in ZK + assertNull(zk.exists(propKey.getPath(), null)); // check node does not exist in ZK propStore.create(propKey, Map.of()); Thread.sleep(25); // yield. - assertNotNull(zooKeeper.exists(propKey.getPath(), false)); // check not created + assertNotNull(zk.exists(propKey.getPath(), null)); // check not created assertThrows(IllegalStateException.class, () -> propStore.create(propKey, null)); assertNotNull(propStore.get(propKey)); @@ -170,7 +165,7 @@ public void createWithProps() throws InterruptedException, KeeperException, IOEx assertEquals("true", vProps.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey())); // check using direct read from ZK - byte[] bytes = zooKeeper.getData(propKey.getPath(), false, new Stat()); + byte[] bytes = zk.getData(propKey.getPath(), null, new Stat()); var readFromZk = propCodec.fromBytes(0, bytes); var propsA = propStore.get(propKey); assertEquals(readFromZk.asMap(), propsA.asMap()); @@ -292,7 +287,7 @@ public void deleteThroughWatcher() throws InterruptedException { assertEquals("true", propsA.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey())); // use alternate prop store - change will propagate via ZooKeeper - PropStore propStore2 = ZooPropStore.initialize(instanceId, context.getZooReaderWriter()); + PropStore propStore2 = ZooPropStore.initialize(instanceId, context.getZooSession()); propStore2.delete(tableAPropKey); @@ -350,8 +345,8 @@ public void externalChange() throws IOException, InterruptedException, KeeperExc byte[] updatedBytes = propCodec.toBytes(pendingProps); // force external write to ZooKeeper - context.getZooReaderWriter().overwritePersistentData(tableAPropKey.getPath(), updatedBytes, - (int) firstRead.getDataVersion()); + context.getZooSession().asReaderWriter().overwritePersistentData(tableAPropKey.getPath(), + updatedBytes, (int) firstRead.getDataVersion()); Thread.sleep(150); diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java index 285790b4b7e..36a7b9adbdf 100644 --- a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java +++ b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java @@ -46,6 +46,8 @@ import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; +import org.apache.accumulo.core.zookeeper.ZooSession.ZKUtil; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.conf.NamespaceConfiguration; import org.apache.accumulo.server.conf.SystemConfiguration; @@ -59,9 +61,7 @@ import org.apache.accumulo.server.conf.store.impl.ZooPropStore; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -82,7 +82,7 @@ public class ZooBasedConfigIT { private static final InstanceId INSTANCE_ID = InstanceId.of(UUID.randomUUID()); private static ZooKeeperTestingServer testZk = null; private static ZooReaderWriter zrw; - private static ZooKeeper zooKeeper; + private static ZooSession zk; private ServerContext context; // fake ids @@ -99,17 +99,15 @@ public class ZooBasedConfigIT { @BeforeAll public static void setupZk() throws Exception { - // using default zookeeper port - we don't have a full configuration testZk = new ZooKeeperTestingServer(tempDir); - zooKeeper = testZk.newClient(); - ZooUtil.digestAuth(zooKeeper, ZooKeeperTestingServer.SECRET); - zrw = testZk.getZooReaderWriter(); + zk = testZk.newClient(); + zrw = zk.asReaderWriter(); } @AfterAll public static void shutdownZK() throws Exception { try { - zooKeeper.close(); + zk.close(); } finally { testZk.close(); } @@ -118,19 +116,20 @@ public static void shutdownZK() throws Exception { @BeforeEach public void initPaths() throws Exception { context = createMock(ServerContext.class); + expect(context.getZooSession()).andReturn(zk); zrw.mkdirs(ZooUtil.getRoot(INSTANCE_ID)); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES, new byte[0], + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tidA.canonical(), + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tidA.canonical(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tidB.canonical(), + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZTABLES + "/" + tidB.canonical(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZNAMESPACES, new byte[0], + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZNAMESPACES, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZNAMESPACES + "/" + nsId.canonical(), + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZNAMESPACES + "/" + nsId.canonical(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); ticker = new TestTicker(); @@ -139,12 +138,10 @@ public void initPaths() throws Exception { // setup context mock with enough to create prop store expect(context.getInstanceID()).andReturn(INSTANCE_ID).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); - expect(context.getZooKeepersSessionTimeOut()).andReturn(zrw.getSessionTimeout()).anyTimes(); replay(context); - propStore = ZooPropStore.initialize(context.getInstanceID(), zrw); + propStore = ZooPropStore.initialize(context.getInstanceID(), zk); reset(context); @@ -153,9 +150,7 @@ public void initPaths() throws Exception { // setup context mock with prop store and the rest of the env needed. expect(context.getInstanceID()).andReturn(INSTANCE_ID).anyTimes(); - expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes(); - expect(context.getZooKeepersSessionTimeOut()).andReturn(zooKeeper.getSessionTimeout()) - .anyTimes(); + expect(context.getZooKeepersSessionTimeOut()).andReturn(zk.getSessionTimeout()).anyTimes(); expect(context.getPropStore()).andReturn(propStore).anyTimes(); expect(context.getSiteConfiguration()).andReturn(SiteConfiguration.empty().build()).anyTimes(); @@ -163,7 +158,7 @@ public void initPaths() throws Exception { @AfterEach public void cleanupZnodes() throws Exception { - ZKUtil.deleteRecursive(zooKeeper, "/accumulo"); + ZKUtil.deleteRecursive(zk, "/accumulo"); verify(context); } @@ -175,7 +170,7 @@ public void cleanupZnodes() throws Exception { public void upgradeSysTestNoProps() throws Exception { replay(context); // force create empty sys config node. - zooKeeper.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZCONFIG, new byte[0], + zk.create(ZooUtil.getRoot(INSTANCE_ID) + Constants.ZCONFIG, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); var propKey = SystemPropKey.of(INSTANCE_ID); ZooBasedConfiguration zbc = new SystemConfiguration(context, propKey, parent); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index 3428349a2cc..be146de65b0 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -58,6 +58,7 @@ import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType; import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.manager.Manager; import org.apache.accumulo.manager.tableOps.ManagerRepo; import org.apache.accumulo.manager.tableOps.TraceRepo; @@ -179,8 +180,9 @@ public Repo call(long tid, Manager environment) throws Exception { @TempDir private static File tempDir; - private static ZooKeeperTestingServer szk = null; - private static ZooReaderWriter zk = null; + private static ZooKeeperTestingServer testZk = null; + private static ZooSession zk = null; + private static ZooReaderWriter zrw = null; private static final InstanceId IID = InstanceId.of(UUID.randomUUID()); private static final String ZK_ROOT = ZooUtil.getRoot(IID); private static final NamespaceId NS = NamespaceId.of("testNameSpace"); @@ -196,18 +198,23 @@ private enum ExceptionLocation { @BeforeAll public static void setup() throws Exception { - szk = new ZooKeeperTestingServer(tempDir); - zk = szk.getZooReaderWriter(); - zk.mkdirs(ZK_ROOT + Constants.ZFATE); - zk.mkdirs(ZK_ROOT + Constants.ZTABLE_LOCKS); - zk.mkdirs(ZK_ROOT + Constants.ZNAMESPACES + "/" + NS.canonical()); - zk.mkdirs(ZK_ROOT + Constants.ZTABLE_STATE + "/" + TID.canonical()); - zk.mkdirs(ZK_ROOT + Constants.ZTABLES + "/" + TID.canonical()); + testZk = new ZooKeeperTestingServer(tempDir); + zk = testZk.newClient(); + zrw = zk.asReaderWriter(); + zrw.mkdirs(ZK_ROOT + Constants.ZFATE); + zrw.mkdirs(ZK_ROOT + Constants.ZTABLE_LOCKS); + zrw.mkdirs(ZK_ROOT + Constants.ZNAMESPACES + "/" + NS.canonical()); + zrw.mkdirs(ZK_ROOT + Constants.ZTABLE_STATE + "/" + TID.canonical()); + zrw.mkdirs(ZK_ROOT + Constants.ZTABLES + "/" + TID.canonical()); } @AfterAll public static void teardown() throws Exception { - szk.close(); + try { + zk.close(); + } finally { + testZk.close(); + } } @Test @@ -221,7 +228,7 @@ public void testTransactionStatus() throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(manager.getContext()).andReturn(sctx).anyTimes(); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(manager, sctx); ConfigurationCopy config = new ConfigurationCopy(); @@ -234,16 +241,16 @@ public void testTransactionStatus() throws Exception { finishCall = new CountDownLatch(1); long txid = fate.startTransaction(); - assertEquals(TStatus.NEW, getTxStatus(zk, txid)); + assertEquals(TStatus.NEW, getTxStatus(zrw, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); - assertEquals(TStatus.SUBMITTED, getTxStatus(zk, txid)); + assertEquals(TStatus.SUBMITTED, getTxStatus(zrw, txid)); // Wait for the transaction runner to be scheduled. Thread.sleep(3000); // wait for call() to be called callStarted.await(); - assertEquals(IN_PROGRESS, getTxStatus(zk, txid)); + assertEquals(IN_PROGRESS, getTxStatus(zrw, txid)); // tell the op to exit the method finishCall.countDown(); // Check that it transitions to SUCCESSFUL and gets removed @@ -251,7 +258,7 @@ public void testTransactionStatus() throws Exception { Wait.waitFor(() -> { TStatus s; try { - switch (s = getTxStatus(zk, txid)) { + switch (s = getTxStatus(zrw, txid)) { case IN_PROGRESS: if (sawSuccess.get()) { fail("Should never see IN_PROGRESS after seeing SUCCESSFUL"); @@ -293,7 +300,7 @@ public void testCancelWhileNew() throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(manager.getContext()).andReturn(sctx).anyTimes(); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(manager, sctx); ConfigurationCopy config = new ConfigurationCopy(); @@ -309,16 +316,16 @@ public void testCancelWhileNew() throws Exception { long txid = fate.startTransaction(); LOG.debug("Starting test testCancelWhileNew with {}", FateTxId.formatTid(txid)); - assertEquals(NEW, getTxStatus(zk, txid)); + assertEquals(NEW, getTxStatus(zrw, txid)); // cancel the transaction assertTrue(fate.cancel(txid)); - assertTrue(FAILED_IN_PROGRESS == getTxStatus(zk, txid) || FAILED == getTxStatus(zk, txid)); + assertTrue(FAILED_IN_PROGRESS == getTxStatus(zrw, txid) || FAILED == getTxStatus(zrw, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); - Wait.waitFor(() -> FAILED == getTxStatus(zk, txid)); + Wait.waitFor(() -> FAILED == getTxStatus(zrw, txid)); // nothing should have run assertEquals(1, callStarted.getCount()); fate.delete(txid); - assertThrows(KeeperException.NoNodeException.class, () -> getTxStatus(zk, txid)); + assertThrows(KeeperException.NoNodeException.class, () -> getTxStatus(zrw, txid)); } finally { fate.shutdown(); } @@ -333,7 +340,7 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(manager.getContext()).andReturn(sctx).anyTimes(); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(manager, sctx); ConfigurationCopy config = new ConfigurationCopy(); @@ -350,17 +357,17 @@ public void testCancelWhileSubmittedAndRunning() throws Exception { long txid = fate.startTransaction(); LOG.debug("Starting test testCancelWhileSubmitted with {}", FateTxId.formatTid(txid)); - assertEquals(NEW, getTxStatus(zk, txid)); + assertEquals(NEW, getTxStatus(zrw, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), false, "Test Op"); - Wait.waitFor(() -> IN_PROGRESS == getTxStatus(zk, txid)); + Wait.waitFor(() -> IN_PROGRESS == getTxStatus(zrw, txid)); // This is false because the transaction runner has reserved the FaTe // transaction. assertFalse(fate.cancel(txid)); callStarted.await(); finishCall.countDown(); - Wait.waitFor(() -> IN_PROGRESS != getTxStatus(zk, txid)); + Wait.waitFor(() -> IN_PROGRESS != getTxStatus(zrw, txid)); fate.delete(txid); - assertThrows(KeeperException.NoNodeException.class, () -> getTxStatus(zk, txid)); + assertThrows(KeeperException.NoNodeException.class, () -> getTxStatus(zrw, txid)); } finally { fate.shutdown(); } @@ -375,7 +382,7 @@ public void testCancelWhileInCall() throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(manager.getContext()).andReturn(sctx).anyTimes(); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(manager, sctx); ConfigurationCopy config = new ConfigurationCopy(); @@ -392,9 +399,9 @@ public void testCancelWhileInCall() throws Exception { long txid = fate.startTransaction(); LOG.debug("Starting test testCancelWhileInCall with {}", FateTxId.formatTid(txid)); - assertEquals(NEW, getTxStatus(zk, txid)); + assertEquals(NEW, getTxStatus(zrw, txid)); fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op"); - assertEquals(SUBMITTED, getTxStatus(zk, txid)); + assertEquals(SUBMITTED, getTxStatus(zrw, txid)); // wait for call() to be called callStarted.await(); @@ -422,7 +429,7 @@ public void testRepoFails() throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(manager.getContext()).andReturn(sctx).anyTimes(); expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(manager, sctx); ConfigurationCopy config = new ConfigurationCopy(); @@ -440,7 +447,7 @@ public void testRepoFails() throws Exception { */ undoLatch = new CountDownLatch(TestOperationFails.TOTAL_NUM_OPS); long txid = fate.startTransaction(); - assertEquals(NEW, getTxStatus(zk, txid)); + assertEquals(NEW, getTxStatus(zrw, txid)); fate.seedTransaction("TestOperationFails", txid, new TestOperationFails(1, ExceptionLocation.CALL), false, "Test Op Fails"); // Wait for all the undo() calls to complete @@ -454,7 +461,7 @@ public void testRepoFails() throws Exception { TestOperationFails.undoOrder = new ArrayList<>(); undoLatch = new CountDownLatch(TestOperationFails.TOTAL_NUM_OPS); txid = fate.startTransaction(); - assertEquals(NEW, getTxStatus(zk, txid)); + assertEquals(NEW, getTxStatus(zrw, txid)); fate.seedTransaction("TestOperationFails", txid, new TestOperationFails(1, ExceptionLocation.IS_READY), false, "Test Op Fails"); // Wait for all the undo() calls to complete diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java index 28cebf2b955..5f197ca1738 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java @@ -81,8 +81,8 @@ public class ZooMutatorIT extends WithTestNames { public void concurrentMutatorTest() throws Exception { File newFolder = new File(tempDir, testName() + "/"); assertTrue(newFolder.isDirectory() || newFolder.mkdir(), "failed to create dir: " + newFolder); - try (ZooKeeperTestingServer szk = new ZooKeeperTestingServer(newFolder)) { - ZooReaderWriter zk = szk.getZooReaderWriter(); + try (var testZk = new ZooKeeperTestingServer(newFolder); var zk = testZk.newClient()) { + var zrw = zk.asReaderWriter(); var executor = Executors.newFixedThreadPool(16); @@ -101,7 +101,7 @@ public void concurrentMutatorTest() throws Exception { int count = -1; while (count < 200) { byte[] val = - zk.mutateOrCreate("/test-zm", initialData.getBytes(UTF_8), this::nextValue); + zrw.mutateOrCreate("/test-zm", initialData.getBytes(UTF_8), this::nextValue); int nextCount = getCount(val); assertTrue(nextCount > count, "nextCount <= count " + nextCount + " " + count); count = nextCount; @@ -120,7 +120,7 @@ public void concurrentMutatorTest() throws Exception { } executor.shutdown(); - byte[] actual = zk.getData("/test-zm"); + byte[] actual = zrw.getData("/test-zm"); int settledCount = getCount(actual); assertTrue(settledCount >= 200); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java index 5e4067f1bbd..dd635d9bd4c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java @@ -45,7 +45,7 @@ public void test() throws Exception { // create a backup Process backup = exec(Manager.class); try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { - ZooReaderWriter writer = getCluster().getServerContext().getZooReaderWriter(); + ZooReaderWriter writer = getServerContext().getZooSession().asReaderWriter(); String root = getCluster().getServerContext().getZooKeeperRoot(); // wait for 2 lock entries @@ -60,7 +60,7 @@ public void test() throws Exception { ServiceLock.validateAndSort(path, writer.getChildren(path.toString())); String lockPath = root + Constants.ZMANAGER_LOCK + "/" + children.get(0); byte[] data = writer.getData(lockPath); - writer.getZooKeeper().setData(lockPath, data, -1); + getServerContext().getZooSession().setData(lockPath, data, -1); // let it propagate Thread.sleep(500); // kill the manager by removing its lock diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java index 70eec7e062e..4ca4a21c29d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java @@ -22,8 +22,8 @@ import java.io.IOException; import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.commons.io.FileUtils; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -36,15 +36,17 @@ public static void main(String[] args) throws Exception { File reportDir = new File(args[1]); var siteConfig = SiteConfiguration.auto(); - ZooReaderWriter zoo = new ZooReaderWriter(siteConfig); + try (var zk = new ZooSession(CacheTestClean.class.getSimpleName(), siteConfig)) { + var zrw = zk.asReaderWriter(); - if (zoo.exists(rootDir)) { - zoo.recursiveDelete(rootDir, NodeMissingPolicy.FAIL); - } + if (zrw.exists(rootDir)) { + zrw.recursiveDelete(rootDir, NodeMissingPolicy.FAIL); + } - FileUtils.deleteQuietly(reportDir); - if (!reportDir.mkdirs()) { - throw new IOException("Unable to (re-)create " + reportDir); + FileUtils.deleteQuietly(reportDir); + if (!reportDir.mkdirs()) { + throw new IOException("Unable to (re-)create " + reportDir); + } } } } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java index be17ed550fb..4d554c5c3e7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java @@ -29,7 +29,7 @@ import java.util.UUID; import org.apache.accumulo.core.fate.zookeeper.ZooCache; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.slf4j.LoggerFactory; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -46,48 +46,49 @@ public static void main(String[] args) throws Exception { File myfile = new File(reportDir + "/" + UUID.randomUUID()); myfile.deleteOnExit(); - ZooCache zc = new ZooCache(new ZooReader(keepers, 30000), null); + try (var zk = new ZooSession(CacheTestReader.class.getSimpleName(), keepers, 30_000, null)) { + ZooCache zc = new ZooCache(zk); - while (true) { - if (myfile.exists() && !myfile.delete()) { - LoggerFactory.getLogger(CacheTestReader.class).warn("Unable to delete {}", myfile); - } + while (true) { + if (myfile.exists() && !myfile.delete()) { + LoggerFactory.getLogger(CacheTestReader.class).warn("Unable to delete {}", myfile); + } - if (zc.get(rootDir + "/die") != null) { - return; - } + if (zc.get(rootDir + "/die") != null) { + return; + } - Map readData = new TreeMap<>(); + Map readData = new TreeMap<>(); - for (int i = 0; i < numData; i++) { - byte[] v = zc.get(rootDir + "/data" + i); - if (v != null) { - readData.put(rootDir + "/data" + i, new String(v, UTF_8)); + for (int i = 0; i < numData; i++) { + byte[] v = zc.get(rootDir + "/data" + i); + if (v != null) { + readData.put(rootDir + "/data" + i, new String(v, UTF_8)); + } } - } - byte[] v = zc.get(rootDir + "/dataS"); - if (v != null) { - readData.put(rootDir + "/dataS", new String(v, UTF_8)); - } + byte[] v = zc.get(rootDir + "/dataS"); + if (v != null) { + readData.put(rootDir + "/dataS", new String(v, UTF_8)); + } - List children = zc.getChildren(rootDir + "/dir"); - if (children != null) { - for (String child : children) { - readData.put(rootDir + "/dir/" + child, ""); + List children = zc.getChildren(rootDir + "/dir"); + if (children != null) { + for (String child : children) { + readData.put(rootDir + "/dir/" + child, ""); + } } - } - FileOutputStream fos = new FileOutputStream(myfile); - ObjectOutputStream oos = new ObjectOutputStream(fos); + FileOutputStream fos = new FileOutputStream(myfile); + ObjectOutputStream oos = new ObjectOutputStream(fos); - oos.writeObject(readData); + oos.writeObject(readData); - fos.close(); - oos.close(); + fos.close(); + oos.close(); - Thread.sleep(20); + Thread.sleep(20); + } } - } } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java index 7bf485f4072..a024f47838c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java @@ -31,9 +31,9 @@ import java.util.UUID; import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -44,127 +44,129 @@ public class CacheTestWriter { @SuppressFBWarnings(value = {"PATH_TRAVERSAL_IN", "OBJECT_DESERIALIZATION"}, justification = "path provided by test; object deserialization is okay for test") public static void main(String[] args) throws Exception { - var zk = new ZooReaderWriter(SiteConfiguration.auto()); - - String rootDir = args[0]; - File reportDir = new File(args[1]); - int numReaders = Integer.parseInt(args[2]); - int numVerifications = Integer.parseInt(args[3]); - int numData = NUM_DATA; + var conf = SiteConfiguration.auto(); + try (var zk = new ZooSession(CacheTestWriter.class.getSimpleName(), conf)) { + var zrw = zk.asReaderWriter(); + + String rootDir = args[0]; + File reportDir = new File(args[1]); + int numReaders = Integer.parseInt(args[2]); + int numVerifications = Integer.parseInt(args[3]); + int numData = NUM_DATA; + + boolean dataSExists = false; + int count = 0; + + zrw.putPersistentData(rootDir, new byte[0], NodeExistsPolicy.FAIL); + for (int i = 0; i < numData; i++) { + zrw.putPersistentData(rootDir + "/data" + i, new byte[0], NodeExistsPolicy.FAIL); + } - boolean dataSExists = false; - int count = 0; + zrw.putPersistentData(rootDir + "/dir", new byte[0], NodeExistsPolicy.FAIL); - zk.putPersistentData(rootDir, new byte[0], NodeExistsPolicy.FAIL); - for (int i = 0; i < numData; i++) { - zk.putPersistentData(rootDir + "/data" + i, new byte[0], NodeExistsPolicy.FAIL); - } + ArrayList children = new ArrayList<>(); - zk.putPersistentData(rootDir + "/dir", new byte[0], NodeExistsPolicy.FAIL); + while (count++ < numVerifications) { - ArrayList children = new ArrayList<>(); + Map expectedData = null; + // change children in dir - while (count++ < numVerifications) { + for (int u = 0; u < RANDOM.get().nextInt(4) + 1; u++) { + expectedData = new TreeMap<>(); - Map expectedData = null; - // change children in dir + if (RANDOM.get().nextFloat() < .5) { + String child = UUID.randomUUID().toString(); + zrw.putPersistentData(rootDir + "/dir/" + child, new byte[0], NodeExistsPolicy.SKIP); + children.add(child); + } else if (!children.isEmpty()) { + int index = RANDOM.get().nextInt(children.size()); + String child = children.remove(index); + zrw.recursiveDelete(rootDir + "/dir/" + child, NodeMissingPolicy.FAIL); + } - for (int u = 0; u < RANDOM.get().nextInt(4) + 1; u++) { - expectedData = new TreeMap<>(); + for (String child : children) { + expectedData.put(rootDir + "/dir/" + child, ""); + } - if (RANDOM.get().nextFloat() < .5) { - String child = UUID.randomUUID().toString(); - zk.putPersistentData(rootDir + "/dir/" + child, new byte[0], NodeExistsPolicy.SKIP); - children.add(child); - } else if (!children.isEmpty()) { - int index = RANDOM.get().nextInt(children.size()); - String child = children.remove(index); - zk.recursiveDelete(rootDir + "/dir/" + child, NodeMissingPolicy.FAIL); - } + // change values + for (int i = 0; i < numData; i++) { + byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8); + zrw.putPersistentData(rootDir + "/data" + i, data, NodeExistsPolicy.OVERWRITE); + expectedData.put(rootDir + "/data" + i, new String(data, UTF_8)); + } - for (String child : children) { - expectedData.put(rootDir + "/dir/" + child, ""); - } + // test a data node that does not always exists... + if (RANDOM.get().nextFloat() < .5) { - // change values - for (int i = 0; i < numData; i++) { - byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8); - zk.putPersistentData(rootDir + "/data" + i, data, NodeExistsPolicy.OVERWRITE); - expectedData.put(rootDir + "/data" + i, new String(data, UTF_8)); - } + byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8); - // test a data node that does not always exists... - if (RANDOM.get().nextFloat() < .5) { + if (dataSExists) { + zrw.putPersistentData(rootDir + "/dataS", data, NodeExistsPolicy.OVERWRITE); + } else { + zrw.putPersistentData(rootDir + "/dataS", data, NodeExistsPolicy.SKIP); + dataSExists = true; + } - byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8); + expectedData.put(rootDir + "/dataS", new String(data, UTF_8)); - if (dataSExists) { - zk.putPersistentData(rootDir + "/dataS", data, NodeExistsPolicy.OVERWRITE); } else { - zk.putPersistentData(rootDir + "/dataS", data, NodeExistsPolicy.SKIP); - dataSExists = true; - } - - expectedData.put(rootDir + "/dataS", new String(data, UTF_8)); - - } else { - if (dataSExists) { - zk.recursiveDelete(rootDir + "/dataS", NodeMissingPolicy.FAIL); - dataSExists = false; + if (dataSExists) { + zrw.recursiveDelete(rootDir + "/dataS", NodeMissingPolicy.FAIL); + dataSExists = false; + } } } - } - // change children in dir and change values + // change children in dir and change values - System.out.println("expectedData " + expectedData); + System.out.println("expectedData " + expectedData); - // wait for all readers to see changes - while (true) { + // wait for all readers to see changes + while (true) { - File[] files = reportDir.listFiles(); - if (files == null) { - throw new IllegalStateException("report directory is inaccessible"); - } + File[] files = reportDir.listFiles(); + if (files == null) { + throw new IllegalStateException("report directory is inaccessible"); + } - System.out.println("files.length " + files.length); + System.out.println("files.length " + files.length); - if (files.length == numReaders) { - boolean ok = true; + if (files.length == numReaders) { + boolean ok = true; - for (File file : files) { - try { - FileInputStream fis = new FileInputStream(file); - ObjectInputStream ois = new ObjectInputStream(fis); + for (File file : files) { + try { + FileInputStream fis = new FileInputStream(file); + ObjectInputStream ois = new ObjectInputStream(fis); - @SuppressWarnings("unchecked") - Map readerMap = (Map) ois.readObject(); + @SuppressWarnings("unchecked") + Map readerMap = (Map) ois.readObject(); - fis.close(); - ois.close(); + fis.close(); + ois.close(); - System.out.println("read " + readerMap); + System.out.println("read " + readerMap); - if (!readerMap.equals(expectedData)) { - System.out.println("maps not equals"); + if (!readerMap.equals(expectedData)) { + System.out.println("maps not equals"); + ok = false; + } + } catch (IOException ioe) { + // log.warn("Failed to read "+files[i], ioe); ok = false; } - } catch (IOException ioe) { - // log.warn("Failed to read "+files[i], ioe); - ok = false; } - } - if (ok) { - break; + if (ok) { + break; + } } - } - Thread.sleep(5); + Thread.sleep(5); + } } - } - zk.putPersistentData(rootDir + "/die", new byte[0], NodeExistsPolicy.FAIL); + zrw.putPersistentData(rootDir + "/die", new byte[0], NodeExistsPolicy.FAIL); + } } - } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 9cdd9d80947..19e4b9accc3 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -41,17 +41,15 @@ import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.clientImpl.ClientContext; -import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.ZooStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.manager.state.tables.TableState; import org.apache.accumulo.harness.AccumuloClusterHarness; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.util.SlowOps; import org.apache.zookeeper.KeeperException; import org.junit.jupiter.api.AfterAll; @@ -80,12 +78,10 @@ public class FateConcurrencyIT extends AccumuloClusterHarness { private static final long SLOW_SCAN_SLEEP_MS = 250L; private AccumuloClient client; - private ClientContext context; + private ServerContext context; private static final ExecutorService pool = Executors.newCachedThreadPool(); - private String secret; - private long maxWaitMillis; private SlowOps slowOps; @@ -98,8 +94,7 @@ protected Duration defaultTimeout() { @BeforeEach public void setup() { client = Accumulo.newClient().from(getClientProps()).build(); - context = (ClientContext) client; - secret = cluster.getSiteConfiguration().get(Property.INSTANCE_SECRET); + context = getServerContext(); maxWaitMillis = Math.max(MINUTES.toMillis(1), defaultTimeout().toMillis() / 2); } @@ -251,7 +246,8 @@ public void getFateStatus() { try { InstanceId instanceId = context.getInstanceID(); - ZooReaderWriter zk = context.getZooReader().asWriter(secret); + + var zk = context.getZooSession(); ZooStore zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); @@ -341,7 +337,7 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep log.trace("tid: {}", tableId); InstanceId instanceId = context.getInstanceID(); - ZooReaderWriter zk = context.getZooReader().asWriter(secret); + var zk = context.getZooSession(); ZooStore zs = new ZooStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk); var lockPath = ServiceLock.path(ZooUtil.getRoot(instanceId) + Constants.ZTABLE_LOCKS + "/" + tableId); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java index 0b099b0a047..2a34064438d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java @@ -59,7 +59,6 @@ import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.AdminUtil.FateStatus; import org.apache.accumulo.core.fate.ZooStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.StoredTabletFile; @@ -228,7 +227,7 @@ private static FateStatus getFateStatus(AccumuloCluster cluster) { try { AdminUtil admin = new AdminUtil<>(false); ServerContext context = cluster.getServerContext(); - ZooReaderWriter zk = context.getZooReaderWriter(); + var zk = context.getZooSession(); ZooStore zs = new ZooStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk); var lockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS); return admin.getStatus(zs, zk, lockPath, null, null); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java index 1e6c36edefc..660d1721d9a 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java @@ -118,7 +118,7 @@ private void killMacGc() throws ProcessNotFoundException, InterruptedException, getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR).iterator().next()); // delete lock in zookeeper if there, this will allow next GC to start quickly var path = ServiceLock.path(getServerContext().getZooKeeperRoot() + Constants.ZGC_LOCK); - ZooReaderWriter zk = getServerContext().getZooReaderWriter(); + ZooReaderWriter zk = getServerContext().getZooSession().asReaderWriter(); try { ServiceLock.deleteLock(zk, path); } catch (IllegalStateException e) { @@ -412,7 +412,7 @@ public void testProperPortAdvertisement() throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { - ZooReaderWriter zk = cluster.getServerContext().getZooReaderWriter(); + ZooReaderWriter zk = cluster.getServerContext().getZooSession().asReaderWriter(); var path = ServiceLock .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK); for (int i = 0; i < 5; i++) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java index 83dd3c83a48..769dda520da 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadServerWatcherIT.java @@ -101,7 +101,7 @@ protected TreeMap splitTablet(Tablet tablet, byte[] splitP final String zooRoot = this.getContext().getZooKeeperRoot(); final String tableZPath = zooRoot + Constants.ZTABLES + "/" + tid.canonical(); try { - this.getContext().getZooReaderWriter().exists(tableZPath, new StuckWatcher()); + this.getContext().getZooSession().asReaderWriter().exists(tableZPath, new StuckWatcher()); } catch (KeeperException | InterruptedException e) { LOG.error("Error setting watch at: {}", tableZPath, e); } @@ -183,7 +183,7 @@ public boolean testTabletServerWithStuckWatcherDies() throws Exception { // Delete the lock for the TabletServer final ServerContext ctx = getServerContext(); final String zooRoot = ctx.getZooKeeperRoot(); - ctx.getZooReaderWriter().recursiveDelete( + ctx.getZooSession().asReaderWriter().recursiveDelete( zooRoot + Constants.ZTSERVERS + "/" + tservers.get(0), NodeMissingPolicy.FAIL); Wait.waitFor(() -> pingServer(client, tservers.get(0)) == false, 60_000); 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 5554441fefd..8001bc77cac 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 @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; @@ -92,28 +93,18 @@ public static void teardown() throws Exception { SharedMiniClusterBase.stopMiniCluster(); } - private ThriftClientTypes.Exec op; + private Function> op; @Test public void testPermissions_setManagerGoalState() throws Exception { // To setManagerGoalState, user needs SystemPermission.SYSTEM - op = client -> { - client.setManagerGoalState(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), + op = user -> client -> { + client.setManagerGoalState(TraceUtil.traceInfo(), user.toThrift(instanceId), ManagerGoalState.NORMAL); return null; }; expectPermissionDenied(op, regularUser); - op = client -> { - client.setManagerGoalState(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), - ManagerGoalState.NORMAL); - return null; - }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.setManagerGoalState(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), - ManagerGoalState.NORMAL); - return null; - }; expectPermissionSuccess(op, privilegedUser); } @@ -139,32 +130,16 @@ public void testPermissions_initiateFlush() throws Exception { tableId = client.tableOperations().tableIdMap().get(tableName); } - op = client -> { - client.initiateFlush(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), tableId); + op = user -> client -> { + client.initiateFlush(TraceUtil.traceInfo(), user.toThrift(instanceId), tableId); return null; }; expectPermissionDenied(op, regularUser); // privileged users can grant themselves permission, but it's not default - op = client -> { - client.initiateFlush(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), tableId); - return null; - }; expectPermissionDenied(op, privilegedUser); - op = client -> { - client.initiateFlush(TraceUtil.traceInfo(), regUserWithWrite.toThrift(instanceId), tableId); - return null; - }; expectPermissionSuccess(op, regUserWithWrite); - op = client -> { - client.initiateFlush(TraceUtil.traceInfo(), regUserWithAlter.toThrift(instanceId), tableId); - return null; - }; expectPermissionSuccess(op, regUserWithAlter); // root user can because they created the table - op = client -> { - client.initiateFlush(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), tableId); - return null; - }; expectPermissionSuccess(op, rootUser); } @@ -191,48 +166,23 @@ public void testPermissions_waitForFlush() throws Exception { } AtomicLong flushId = new AtomicLong(); // initiateFlush as the root user to get the flushId, then test waitForFlush with other users - op = client -> { - flushId - .set(client.initiateFlush(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), tableId)); + op = user -> client -> { + flushId.set(client.initiateFlush(TraceUtil.traceInfo(), user.toThrift(instanceId), tableId)); return null; }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.waitForFlush(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), tableId, + op = user -> client -> { + client.waitForFlush(TraceUtil.traceInfo(), user.toThrift(instanceId), tableId, TextUtil.getByteBuffer(new Text("myrow")), TextUtil.getByteBuffer(new Text("myrow~")), flushId.get(), 1); return null; }; expectPermissionDenied(op, regularUser); // privileged users can grant themselves permission, but it's not default - op = client -> { - client.waitForFlush(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), tableId, - TextUtil.getByteBuffer(new Text("myrow")), TextUtil.getByteBuffer(new Text("myrow~")), - flushId.get(), 1); - return null; - }; expectPermissionDenied(op, privilegedUser); - op = client -> { - client.waitForFlush(TraceUtil.traceInfo(), regUserWithWrite.toThrift(instanceId), tableId, - TextUtil.getByteBuffer(new Text("myrow")), TextUtil.getByteBuffer(new Text("myrow~")), - flushId.get(), 1); - return null; - }; expectPermissionSuccess(op, regUserWithWrite); - op = client -> { - client.waitForFlush(TraceUtil.traceInfo(), regUserWithAlter.toThrift(instanceId), tableId, - TextUtil.getByteBuffer(new Text("myrow")), TextUtil.getByteBuffer(new Text("myrow~")), - flushId.get(), 1); - return null; - }; expectPermissionSuccess(op, regUserWithAlter); // root user can because they created the table - op = client -> { - client.waitForFlush(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), tableId, - TextUtil.getByteBuffer(new Text("myrow")), TextUtil.getByteBuffer(new Text("myrow~")), - flushId.get(), 1); - return null; - }; expectPermissionSuccess(op, rootUser); } @@ -240,20 +190,15 @@ public void testPermissions_waitForFlush() throws Exception { public void testPermissions_modifySystemProperties() throws Exception { // To setSystemProperty, user needs SystemPermission.SYSTEM String propKey = Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(); - op = client -> { - client.modifySystemProperties(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), + op = user -> client -> { + client.modifySystemProperties(TraceUtil.traceInfo(), user.toThrift(instanceId), new TVersionedProperties(0, Map.of(propKey, "10000"))); return null; }; expectPermissionDenied(op, regularUser); - op = client -> { - client.modifySystemProperties(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), - new TVersionedProperties(0, Map.of(propKey, "10000"))); - return null; - }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.modifySystemProperties(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), + op = user -> client -> { + client.modifySystemProperties(TraceUtil.traceInfo(), user.toThrift(instanceId), new TVersionedProperties(1, Map.of(propKey, "10000"))); return null; }; @@ -272,20 +217,14 @@ public void testPermissions_removeSystemProperty() throws Exception { client.instanceOperations().setProperty(propKey1, "10000"); // ensure it exists client.instanceOperations().setProperty(propKey2, "10000"); // ensure it exists } - op = client -> { - client.removeSystemProperty(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), - propKey1); + op = user -> client -> { + client.removeSystemProperty(TraceUtil.traceInfo(), user.toThrift(instanceId), propKey1); return null; }; expectPermissionDenied(op, regularUser); - op = client -> { - client.removeSystemProperty(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), propKey1); - return null; - }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.removeSystemProperty(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), - propKey2); + op = user -> client -> { + client.removeSystemProperty(TraceUtil.traceInfo(), user.toThrift(instanceId), propKey2); return null; }; expectPermissionSuccess(op, privilegedUser); @@ -295,23 +234,12 @@ public void testPermissions_removeSystemProperty() throws Exception { public void testPermissions_setSystemProperty() throws Exception { // To setSystemProperty, user needs SystemPermission.SYSTEM String propKey = Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(); - op = client -> { - client.setSystemProperty(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), propKey, - "10000"); + op = user -> client -> { + client.setSystemProperty(TraceUtil.traceInfo(), user.toThrift(instanceId), propKey, "10000"); return null; }; expectPermissionDenied(op, regularUser); - op = client -> { - client.setSystemProperty(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), propKey, - "10000"); - return null; - }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.setSystemProperty(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), propKey, - "10000"); - return null; - }; expectPermissionSuccess(op, privilegedUser); try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { client.instanceOperations().removeProperty(propKey); // clean up property @@ -323,37 +251,27 @@ public void testPermissions_shutdownTabletServer() throws Exception { // To shutdownTabletServer, user needs SystemPermission.SYSTEM // this server won't exist, so shutting it down is a NOOP on success String fakeHostAndPort = getUniqueNames(1)[0] + ":0"; - op = client -> { - client.shutdownTabletServer(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), - fakeHostAndPort, false); + op = user -> client -> { + client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), fakeHostAndPort, + false); return null; }; expectPermissionDenied(op, regularUser); - op = client -> { - client.shutdownTabletServer(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), - fakeHostAndPort, false); - return null; - }; expectPermissionSuccess(op, rootUser); - op = client -> { - client.shutdownTabletServer(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), - fakeHostAndPort, false); - return null; - }; expectPermissionSuccess(op, privilegedUser); } @Test public void shutdownTabletServer() throws Exception { - op = client -> { - client.shutdownTabletServer(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), + op = user -> client -> { + client.shutdownTabletServer(TraceUtil.traceInfo(), user.toThrift(instanceId), "fakeTabletServer:9997", true); return null; }; try (AccumuloClient client = Accumulo.newClient().from(getClientProps()) .as(rootUser.getPrincipal(), rootUser.getToken()).build()) { ClientContext context = (ClientContext) client; - ThriftClientTypes.MANAGER.execute(context, op); + ThriftClientTypes.MANAGER.execute(context, op.apply(rootUser)); } } @@ -368,32 +286,29 @@ public void z99_testPermissions_shutdown() throws Exception { .as(privilegedUser.getPrincipal(), privilegedUser.getToken()); try (var rootClient = rootUserBuilder.build(); var privClient = privUserBuilder.build()) { // To shutdown, user needs SystemPermission.SYSTEM - op = client -> { - client.shutdown(TraceUtil.traceInfo(), regularUser.toThrift(instanceId), false); + op = user -> client -> { + client.shutdown(TraceUtil.traceInfo(), user.toThrift(instanceId), false); return null; }; expectPermissionDenied(op, regularUser); - // We should be able to do both of the following RPC calls before it actually shuts down - op = client -> { - client.shutdown(TraceUtil.traceInfo(), rootUser.toThrift(instanceId), false); - return null; - }; - expectPermissionSuccess(op, (ClientContext) rootClient); - op = client -> { - client.shutdown(TraceUtil.traceInfo(), privilegedUser.toThrift(instanceId), false); - return null; - }; - expectPermissionSuccess(op, (ClientContext) privClient); + expectPermissionSuccess(op.apply(rootUser), (ClientContext) rootClient); + + // make sure it's stopped, then start it again to test with the privileged user + getCluster().stop(); + getCluster().start(); + + // make sure regular user is still denied after restart + expectPermissionDenied(op, regularUser); + expectPermissionSuccess(op.apply(privilegedUser), (ClientContext) privClient); } } private static void expectPermissionSuccess( - ThriftClientTypes.Exec op, Credentials user) - throws Exception { + Function> op, + Credentials user) throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProps()) .as(user.getPrincipal(), user.getToken()).build()) { - ClientContext context = (ClientContext) client; - ThriftClientTypes.MANAGER.execute(context, op); + ThriftClientTypes.MANAGER.execute((ClientContext) client, op.apply(user)); } } @@ -404,9 +319,9 @@ private static void expectPermissionSuccess( } private static void expectPermissionDenied( - ThriftClientTypes.Exec op, Credentials user) { - AccumuloSecurityException e = - assertThrows(AccumuloSecurityException.class, () -> expectPermissionSuccess(op, user)); + Function> op, + Credentials user) { + var e = assertThrows(AccumuloSecurityException.class, () -> expectPermissionSuccess(op, user)); assertSame(SecurityErrorCode.PERMISSION_DENIED, e.getSecurityErrorCode()); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index a9119a48c0b..afdec2e6743 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -95,9 +95,9 @@ private KeyExtent nke(String table, String endRow, String prevEndRow) { private void run(ServerContext c) throws Exception { var zPath = ServiceLock.path(c.getZooKeeperRoot() + "/testLock"); - ZooReaderWriter zoo = c.getZooReaderWriter(); + ZooReaderWriter zoo = c.getZooSession().asReaderWriter(); zoo.putPersistentData(zPath.toString(), new byte[0], NodeExistsPolicy.OVERWRITE); - ServiceLock zl = new ServiceLock(zoo.getZooKeeper(), zPath, UUID.randomUUID()); + ServiceLock zl = new ServiceLock(c.getZooSession(), zPath, UUID.randomUUID()); boolean gotLock = zl.tryLock(new LockWatcher() { @SuppressFBWarnings(value = "DM_EXIT", diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java index c8c01d2f733..77ac8a1b869 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java @@ -64,8 +64,8 @@ public void test() throws Exception { } c.tableOperations().list(); String zooKeepers = ClientProperty.INSTANCE_ZOOKEEPERS.getValue(props); - // base number of watchers 90 to 125, and 15 to 25 per table in a single-node instance. - final long MIN = 125L; + // expect about 30-45 base + about 12 per table in a single-node 3-tserver instance + final long MIN = 75L; final long MAX = 200L; long total = 0; final HostAndPort hostAndPort = HostAndPort.fromString(zooKeepers); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java index 051efd6c1f2..3a352cc7a6d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java @@ -135,10 +135,10 @@ public static void main(String[] args) throws Exception { var zLockPath = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + addressString); - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP); - ServiceLock zlock = new ServiceLock(zoo.getZooKeeper(), zLockPath, UUID.randomUUID()); + ServiceLock zlock = new ServiceLock(context.getZooSession(), zLockPath, UUID.randomUUID()); MetricsInfo metricsInfo = context.getMetricsInfo(); metricsInfo.init(MetricsInfo.serviceTags(context.getInstanceName(), "zombie.server", diff --git a/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java index 0e51f5056d8..60e9ed867bd 100644 --- a/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java +++ b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java @@ -38,7 +38,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.LockSupport; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.AccumuloLockWatcher; import org.apache.accumulo.core.lock.ServiceLock.LockLossReason; @@ -46,12 +45,11 @@ import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -70,23 +68,29 @@ public class ServiceLockIT { @TempDir private static File tempDir; - private static ZooKeeperTestingServer szk = null; + private static ZooKeeperTestingServer testZk = null; + private static ZooSession zk = null; @BeforeAll public static void setup() throws Exception { - szk = new ZooKeeperTestingServer(tempDir); + testZk = new ZooKeeperTestingServer(tempDir); + zk = testZk.newClient(); } @AfterAll public static void teardown() throws Exception { - szk.close(); + try { + zk.close(); + } finally { + testZk.close(); + } } - static class ZooKeeperWrapper extends ZooKeeper { + static class ZooKeeperWrapper extends ZooSession { - public ZooKeeperWrapper(String connectString, int sessionTimeout, Watcher watcher) - throws IOException { - super(connectString, sessionTimeout, watcher); + public ZooKeeperWrapper(String clientName, String connectString, int sessionTimeout, + String auth) throws IOException { + super(clientName, connectString, sessionTimeout, auth); } public void createOnce(String path, byte[] data, List acl, CreateMode createMode) @@ -177,7 +181,7 @@ public synchronized void unableToMonitorLockNode(Exception e) { private ServiceLock getZooLock(ServiceLockPath parent, UUID randomUUID) throws IOException, InterruptedException { - return new ServiceLock(szk.newClient(), parent, randomUUID); + return new ServiceLock(testZk.newClient(), parent, randomUUID); } private static ServiceLock getZooLock(ZooKeeperWrapper zkw, ServiceLockPath parent, UUID uuid) { @@ -195,14 +199,14 @@ public void testDeleteParent() throws Exception { assertFalse(zl.isLocked()); assertFalse(zl.verifyLockAtSource()); - ZooReaderWriter zk = szk.getZooReaderWriter(); + var zrw = zk.asReaderWriter(); // intentionally created parent after lock - zk.mkdirs(parent.toString()); + zrw.mkdirs(parent.toString()); - zk.delete(parent.toString()); + zrw.delete(parent.toString()); - zk.mkdirs(parent.toString()); + zrw.mkdirs(parent.toString()); TestALW lw = new TestALW(); @@ -252,8 +256,8 @@ public void testDeleteLock() throws Exception { var parent = ServiceLock.path("/zltestDeleteLock-" + this.hashCode() + "-l" + pdCount.incrementAndGet()); - ZooReaderWriter zk = szk.getZooReaderWriter(); - zk.mkdirs(parent.toString()); + var zrw = zk.asReaderWriter(); + zrw.mkdirs(parent.toString()); ServiceLock zl = getZooLock(parent, UUID.randomUUID()); @@ -273,7 +277,7 @@ public void testDeleteLock() throws Exception { assertNull(lw.exception); assertNull(lw.reason); - zk.delete(zl.getLockPath()); + zrw.delete(zl.getLockPath()); lw.waitForChanges(2); @@ -288,8 +292,8 @@ public void testDeleteWaiting() throws Exception { var parent = ServiceLock .path("/zltestDeleteWaiting-" + this.hashCode() + "-l" + pdCount.incrementAndGet()); - ZooReaderWriter zk = szk.getZooReaderWriter(); - zk.mkdirs(parent.toString()); + var zrw = zk.asReaderWriter(); + zrw.mkdirs(parent.toString()); ServiceLock zl = getZooLock(parent, UUID.randomUUID()); @@ -327,9 +331,9 @@ public void testDeleteWaiting() throws Exception { zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), "test3", ThriftService.TSERV, ServiceDescriptor.DEFAULT_GROUP_NAME)); - List children = ServiceLock.validateAndSort(parent, zk.getChildren(parent.toString())); + List children = ServiceLock.validateAndSort(parent, zrw.getChildren(parent.toString())); - zk.delete(parent + "/" + children.get(1)); + zrw.delete(parent + "/" + children.get(1)); lw2.waitForChanges(1); @@ -337,7 +341,7 @@ public void testDeleteWaiting() throws Exception { assertNotNull(lw2.exception); assertNull(lw2.reason); - zk.delete(parent + "/" + children.get(0)); + zrw.delete(parent + "/" + children.get(0)); lw.waitForChanges(2); @@ -363,7 +367,7 @@ public void testUnexpectedEvent() throws Exception { var parent = ServiceLock .path("/zltestUnexpectedEvent-" + this.hashCode() + "-l" + pdCount.incrementAndGet()); - try (var zk = szk.newClient()) { + try (var zk = testZk.newClient()) { zk.create(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -407,8 +411,8 @@ public void testUnexpectedEvent() throws Exception { public void testLockSerial() throws Exception { var parent = ServiceLock.path("/zlretryLockSerial"); - try (ZooKeeperWrapper zk1 = szk.newClient(ZooKeeperWrapper::new); - ZooKeeperWrapper zk2 = szk.newClient(ZooKeeperWrapper::new)) { + try (ZooKeeperWrapper zk1 = testZk.newClient(ZooKeeperWrapper::new); + ZooKeeperWrapper zk2 = testZk.newClient(ZooKeeperWrapper::new)) { // Create the parent node zk1.createOnce(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, @@ -452,7 +456,7 @@ public void testLockSerial() throws Exception { assertTrue(zlw1.isLockHeld()); assertFalse(zlw2.isLockHeld()); - List children = zk1.getChildren(parent.toString(), false); + List children = zk1.getChildren(parent.toString(), null); assertTrue(children.contains("zlock#00000000-0000-0000-0000-aaaaaaaaaaaa#0000000000")); assertFalse(children.contains("zlock#00000000-0000-0000-0000-aaaaaaaaaaaa#0000000001"), "this node should have been deleted"); @@ -515,17 +519,15 @@ public boolean holdsLock() { @Override public void run() { - try { - try (ZooKeeperWrapper zk = szk.newClient(ZooKeeperWrapper::new)) { - ServiceLock zl = getZooLock(zk, parent, uuid); - getLockLatch.countDown(); // signal we are done - getLockLatch.await(); // wait for others to finish - zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, - ServiceDescriptor.DEFAULT_GROUP_NAME)); // race to the lock - lockCompletedLatch.countDown(); - unlockLatch.await(); - zl.unlock(); - } + try (ZooKeeperWrapper zk = testZk.newClient(ZooKeeperWrapper::new)) { + ServiceLock zl = getZooLock(zk, parent, uuid); + getLockLatch.countDown(); // signal we are done + getLockLatch.await(); // wait for others to finish + zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV, + ServiceDescriptor.DEFAULT_GROUP_NAME)); // race to the lock + lockCompletedLatch.countDown(); + unlockLatch.await(); + zl.unlock(); } catch (Exception e) { LOG.error("Error in LockWorker.run() for {}", uuid, e); ex = e; @@ -562,7 +564,7 @@ private int parseLockWorkerName(String child) { public void testLockParallel() throws Exception { var parent = ServiceLock.path("/zlParallel"); - try (ZooKeeperWrapper zk = szk.newClient(ZooKeeperWrapper::new)) { + try (ZooKeeperWrapper zk = testZk.newClient(ZooKeeperWrapper::new)) { // Create the parent node zk.createOnce(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -593,7 +595,7 @@ public void testLockParallel() throws Exception { ServiceLock.validateAndSort(parent, zk.getChildren(parent.toString(), null)); while (children.size() != i) { Thread.sleep(100); - children = zk.getChildren(parent.toString(), false); + children = zk.getChildren(parent.toString(), null); } assertEquals(i, children.size()); String first = children.get(0); @@ -612,7 +614,7 @@ public void testLockParallel() throws Exception { workers.forEach(w -> assertFalse(w.holdsLock())); workers.forEach(w -> assertNull(w.getException())); - assertEquals(0, zk.getChildren(parent.toString(), false).size()); + assertEquals(0, zk.getChildren(parent.toString(), null).size()); threads.forEach(Uninterruptibles::joinUninterruptibly); } @@ -627,7 +629,7 @@ public void testTryLock() throws Exception { ServiceLock zl = getZooLock(parent, UUID.randomUUID()); - try (var zk = szk.newClient()) { + try (var zk = testZk.newClient()) { for (int i = 0; i < 10; i++) { zk.create(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, @@ -661,7 +663,7 @@ public void testChangeData() throws Exception { var parent = ServiceLock.path("/zltestChangeData-" + this.hashCode() + "-l" + pdCount.incrementAndGet()); - try (var zk = szk.newClient()) { + try (var zk = testZk.newClient()) { zk.create(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); ServiceLock zl = getZooLock(parent, UUID.randomUUID()); diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java index c52c5a74f01..d36c9919fce 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java @@ -314,7 +314,7 @@ public static void main(String[] args) throws Exception { int zkTimeOut = (int) DefaultConfiguration.getInstance().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT); var siteConfig = SiteConfiguration.auto(); - ServerContext context = ServerContext.override(siteConfig, opts.iname, opts.keepers, zkTimeOut); + var context = ServerContext.forTesting(siteConfig, opts.iname, opts.keepers, zkTimeOut); ClientServiceHandler csh = new ClientServiceHandler(context, new TransactionWatcher(context)); NullTServerTabletClientHandler tch = new NullTServerTabletClientHandler(); diff --git a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java index ae352ff2354..f659ea97971 100644 --- a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java +++ b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java @@ -20,15 +20,10 @@ import java.io.File; import java.io.IOException; -import java.util.concurrent.CountDownLatch; -import org.apache.accumulo.core.fate.zookeeper.ZooReader; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.util.PortUtils; import org.apache.curator.test.TestingServer; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,8 +69,8 @@ public ZooKeeperTestingServer(final File tmpDir) { } @FunctionalInterface - public interface ZooKeeperConstructor { - public T construct(String connectString, int sessionTimeout, Watcher watcher) + public interface ZooSessionConstructor { + public T construct(String clientName, String connectString, int timeout, String instanceSecret) throws IOException; } @@ -85,29 +80,18 @@ public T construct(String connectString, int sessionTimeout, Watcher watcher) * used by this class to wait for the client to connect. This can be used to construct a subclass * of the ZooKeeper client that implements non-standard behavior for a test. */ - public T newClient(ZooKeeperConstructor f) + public T newClient(ZooSessionConstructor f) throws IOException, InterruptedException { - var connectionLatch = new CountDownLatch(1); - var zoo = f.construct(zkServer.getConnectString(), 30_000, watchedEvent -> { - if (watchedEvent.getState() == Watcher.Event.KeeperState.SyncConnected) { - connectionLatch.countDown(); - } - }); - connectionLatch.await(); - ZooUtil.digestAuth(zoo, SECRET); - return zoo; + return f.construct(ZooKeeperTestingServer.class.getSimpleName(), zkServer.getConnectString(), + 30_000, SECRET); } /** * Create a new instance of a standard ZooKeeper client that is already connected to the testing - * server. + * server. The caller is responsible for closing the object. */ - public ZooKeeper newClient() throws IOException, InterruptedException { - return newClient(ZooKeeper::new); - } - - public ZooReaderWriter getZooReaderWriter() { - return new ZooReader(zkServer.getConnectString(), 30000).asWriter(SECRET); + public ZooSession newClient() throws IOException, InterruptedException { + return newClient(ZooSession::new); } @Override