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 3c93a6d39e2..966ca3b62e3 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.util.threads.ThreadPoolNames.CONDITIONAL_WRITER_CLEANUP_POOL; @@ -44,11 +43,11 @@ 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; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloClient; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; @@ -79,8 +78,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; @@ -105,6 +102,7 @@ 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; @@ -127,14 +125,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 @@ -162,6 +158,9 @@ public class ClientContext implements AccumuloClient { private MeterRegistry micrometer; private Caches caches; + private final AtomicBoolean zooKeeperOpened = new AtomicBoolean(false); + private final Supplier zooSession; + private void ensureOpen() { if (closed) { throw new IllegalStateException("This client was closed."); @@ -228,13 +227,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); @@ -333,7 +338,7 @@ public synchronized void setCredentials(Credentials newCredentials) { */ public AccumuloConfiguration getConfiguration() { ensureOpen(); - return serverConf; + return accumuloConf; } /** @@ -484,26 +489,7 @@ public synchronized TCredentials rpcCreds() { * @return a UUID */ public InstanceId getInstanceID() { - 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() { @@ -541,7 +527,7 @@ public int getZooKeepersSessionTimeOut() { } public ZooCache getZooCache() { - return zooCache; + return zooCache.get(); } private TableZooHelper tableZooHelper; @@ -796,6 +782,9 @@ public AuthenticationToken token() { @Override public synchronized void close() { closed = true; + if (zooKeeperOpened.get()) { + zooSession.get().close(); + } if (thriftTransportPool != null) { thriftTransportPool.shutdown(); } @@ -816,7 +805,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; @@ -825,12 +814,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() { @@ -1001,7 +987,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; } @@ -1025,9 +1011,9 @@ public ClientFactory withUncaughtExceptionHandler(UncaughtExceptionHandler ue } - public ZooReader getZooReader() { + public ZooSession getZooSession() { ensureOpen(); - return zooReader; + return zooSession.get(); } protected long getTransportPoolMaxAgeMillis() { @@ -1069,7 +1055,14 @@ && getConfiguration().getBoolean(Property.GENERAL_MICROMETER_CACHE_METRICS_ENABL 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(), getServerPaths()); } 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 9b260d2ba05..7c22b7d8a87 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 @@ -19,7 +19,6 @@ package org.apache.accumulo.core.clientImpl; import static com.google.common.base.Preconditions.checkArgument; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.rpc.ThriftUtil.createClient; import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport; import static org.apache.accumulo.core.rpc.ThriftUtil.getClient; @@ -60,7 +59,6 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException; import org.apache.accumulo.core.conf.DeprecatedPropertyUtil; import org.apache.accumulo.core.data.InstanceId; -import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; @@ -467,22 +465,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 e3b212ce72b..cd9c3b7de73 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,11 +18,15 @@ */ package org.apache.accumulo.core.clientImpl; +import static java.util.Objects.requireNonNull; + import java.util.Set; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.clientImpl.ClientTabletCacheImpl.TabletServerLockChecker; +import org.apache.accumulo.core.fate.zookeeper.ZooCache; import org.apache.accumulo.core.lock.ServiceLock; +import org.apache.accumulo.core.lock.ServiceLockPaths; import org.apache.accumulo.core.lock.ServiceLockPaths.AddressSelector; import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; @@ -30,17 +34,21 @@ public class ZookeeperLockChecker implements TabletServerLockChecker { - private final ClientContext ctx; + private final ZooCache zc; + private final String root; + private final ServiceLockPaths lockPaths; - ZookeeperLockChecker(ClientContext context) { - this.ctx = context; + ZookeeperLockChecker(ZooCache zooCache, String zkRoot, ServiceLockPaths serviceLockPaths) { + this.zc = requireNonNull(zooCache); + this.root = requireNonNull(zkRoot); + this.lockPaths = requireNonNull(serviceLockPaths); } public boolean doesTabletServerLockExist(String server) { // ServiceLockPaths only returns items that have a lock var hostAndPort = HostAndPort.fromString(server); Set tservers = - ctx.getServerPaths().getTabletServer(rg -> true, AddressSelector.exact(hostAndPort), true); + lockPaths.getTabletServer(rg -> true, AddressSelector.exact(hostAndPort), true); return !tservers.isEmpty(); } @@ -49,9 +57,9 @@ public boolean isLockHeld(String server, String session) { // ServiceLockPaths only returns items that have a lock var hostAndPort = HostAndPort.fromString(server); Set tservers = - ctx.getServerPaths().getTabletServer(rg -> true, AddressSelector.exact(hostAndPort), true); + lockPaths.getTabletServer(rg -> true, AddressSelector.exact(hostAndPort), true); for (ServiceLockPath slp : tservers) { - if (ServiceLock.getSessionId(ctx.getZooCache(), slp) == Long.parseLong(session, 16)) { + if (ServiceLock.getSessionId(zc, slp) == Long.parseLong(session, 16)) { return true; } } @@ -62,7 +70,6 @@ public boolean isLockHeld(String server, String session) { public void invalidateCache(String tserver) { // The path for the tserver contains a resource group. The resource group is unknown, so can not // construct a prefix. Therefore clear any path that contains the tserver. - ctx.getZooCache().clear(path -> path.startsWith(ctx.getZooKeeperRoot() + Constants.ZTSERVERS) - && path.contains(tserver)); + zc.clear(path -> path.startsWith(root + Constants.ZTSERVERS) && path.contains(tserver)); } } 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 384099a832e..54fb62e6a5a 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 @@ -40,11 +40,10 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore.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.ServiceLockPaths.ServiceLockPath; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -202,7 +201,7 @@ public Map> getDanglingWaitingLocks() { /** * Returns a list of the FATE transactions, optionally filtered by fate id, status, and fate * instance type. This method does not process lock information, if lock information is desired, - * use {@link #getStatus(ReadOnlyFateStore, ZooReader, ServiceLockPath, Set, EnumSet, EnumSet)} + * use {@link #getStatus(ReadOnlyFateStore, ZooSession, ServiceLockPath, Set, EnumSet, EnumSet)} * * @param fateStores read-only fate stores * @param fateIdFilter filter results to include only provided fate transaction ids @@ -234,7 +233,7 @@ public List getTransactionStatus( * @throws KeeperException if zookeeper exception occurs * @throws InterruptedException if process is interrupted. */ - public FateStatus getStatus(ReadOnlyFateStore mfs, ZooReader zk, ServiceLockPath lockPath, + public FateStatus getStatus(ReadOnlyFateStore mfs, ZooSession zk, ServiceLockPath lockPath, Set fateIdFilter, EnumSet statusFilter, EnumSet typesFilter) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); @@ -254,7 +253,7 @@ public FateStatus getStatus(ReadOnlyFateStore ufs, Set fateIdFilter, typesFilter, new HashMap<>(), new HashMap<>()); } - public FateStatus getStatus(Map> fateStores, ZooReader zk, + public FateStatus getStatus(Map> fateStores, ZooSession zk, ServiceLockPath lockPath, Set fateIdFilter, EnumSet statusFilter, EnumSet typesFilter) throws KeeperException, InterruptedException { Map> heldLocks = new HashMap<>(); @@ -269,19 +268,21 @@ public FateStatus getStatus(Map> fateStore /** * 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 ServiceLockPath lockPath, + private void findLocks(ZooSession zk, final 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) { @@ -289,14 +290,14 @@ private void findLocks(ZooReader zk, final 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); // Example data: "READ:". FateId contains ':' hence the limit of 2 String[] lda = new String(data, UTF_8).split(":", 2); FateId fateId = FateId.from(lda[1]); @@ -412,12 +413,12 @@ private static boolean includeByInstanceType(FateInstanceType type, return typesFilter == null || typesFilter.isEmpty() || typesFilter.contains(type); } - public void printAll(Map> fateStores, ZooReader zk, + public void printAll(Map> fateStores, ZooSession zk, ServiceLockPath tableLocksPath) throws KeeperException, InterruptedException { print(fateStores, zk, tableLocksPath, new Formatter(System.out), null, null, null); } - public void print(Map> fateStores, ZooReader zk, + public void print(Map> fateStores, ZooSession zk, ServiceLockPath tableLocksPath, Formatter fmt, Set fateIdFilter, EnumSet statusFilter, EnumSet typesFilter) throws KeeperException, InterruptedException { @@ -433,7 +434,7 @@ public void print(Map> fateStores, ZooRead fmt.format(" %s transactions", fateStatus.getTransactions().size()); } - public boolean prepDelete(Map> stores, ZooReaderWriter zk, + public boolean prepDelete(Map> stores, ZooSession zk, ServiceLockPath path, String fateIdStr) { if (!checkGlobalLock(zk, path)) { return false; @@ -476,7 +477,7 @@ public boolean prepDelete(Map> stores, ZooReaderWr return state; } - public boolean prepFail(Map> stores, ZooReaderWriter zk, + public boolean prepFail(Map> stores, ZooSession zk, ServiceLockPath zLockManagerPath, String fateIdStr) { if (!checkGlobalLock(zk, zLockManagerPath)) { return false; @@ -527,20 +528,22 @@ public boolean prepFail(Map> stores, ZooReaderWrit return state; } - public void deleteLocks(ZooReaderWriter zk, ServiceLockPath path, String fateIdStr) + public void deleteLocks(ZooSession zk, ServiceLockPath path, String fateIdStr) 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); // Example data: "READ:". FateId contains ':' hence the limit of 2 String[] lda = new String(data, UTF_8).split(":", 2); if (lda[1].equals(fateIdStr)) { - zk.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); } } } @@ -549,9 +552,9 @@ public void deleteLocks(ZooReaderWriter zk, ServiceLockPath path, String fateIdS @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).isPresent()) { + if (ServiceLock.getLockData(zk, zLockManagerPath).isPresent()) { 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/zookeeper/MetaFateStore.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java index e639ac57120..148292ed745 100644 --- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java +++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/MetaFateStore.java @@ -56,6 +56,7 @@ import org.apache.accumulo.core.fate.Repo; import org.apache.accumulo.core.fate.StackOverflowException; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.hadoop.io.DataInputBuffer; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -75,26 +76,28 @@ public class MetaFateStore extends AbstractFateStore { private static final Logger log = LoggerFactory.getLogger(MetaFateStore.class); private static final FateInstanceType fateInstanceType = FateInstanceType.META; private String path; - private ZooReaderWriter zk; + private ZooSession zk; + private ZooReaderWriter zrw; private String getTXPath(FateId fateId) { return path + "/tx_" + fateId.getTxUUIDStr(); } - public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, + public MetaFateStore(String path, ZooSession zk, ZooUtil.LockID lockID, Predicate isLockHeld) throws KeeperException, InterruptedException { this(path, zk, lockID, isLockHeld, DEFAULT_MAX_DEFERRED, DEFAULT_FATE_ID_GENERATOR); } @VisibleForTesting - public MetaFateStore(String path, ZooReaderWriter zk, ZooUtil.LockID lockID, + public MetaFateStore(String path, ZooSession zk, ZooUtil.LockID lockID, Predicate isLockHeld, int maxDeferred, FateIdGenerator fateIdGenerator) throws KeeperException, InterruptedException { super(lockID, isLockHeld, maxDeferred, fateIdGenerator); this.path = path; this.zk = zk; + this.zrw = zk.asReaderWriter(); - zk.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); + this.zrw.putPersistentData(path, new byte[0], NodeExistsPolicy.SKIP); } /** @@ -107,7 +110,7 @@ public FateId create() { while (true) { try { FateId fateId = fateIdGenerator.newRandomId(fateInstanceType); - zk.putPersistentData(getTXPath(fateId), + zrw.putPersistentData(getTXPath(fateId), new FateData(TStatus.NEW, null, null, createEmptyRepoDeque(), createEmptyTxInfo()) .serialize(), NodeExistsPolicy.FAIL); @@ -126,7 +129,7 @@ private Optional> createAndReserve(FateKey fateKey) { try { byte[] newSerFateData = - zk.mutateOrCreate(getTXPath(fateId), new FateData<>(TStatus.NEW, reservation, fateKey, + zrw.mutateOrCreate(getTXPath(fateId), new FateData<>(TStatus.NEW, reservation, fateKey, createEmptyRepoDeque(), createEmptyTxInfo()).serialize(), currSerFateData -> { // We are only returning a non-null value for the following cases: // 1) The existing node for fateId is exactly the same as the value set for the @@ -426,7 +429,7 @@ private void _delete(Set requiredStatus) { "Tried to delete fate data for %s when the transaction status is %s", fateId, fateData.status); try { - zk.deleteStrict(getTXPath(fateId), stat.getVersion()); + zrw.deleteStrict(getTXPath(fateId), stat.getVersion()); this.deleted = true; } catch (KeeperException.BadVersionException e) { log.trace( @@ -466,7 +469,7 @@ public long timeCreated() { verifyReservedAndNotDeleted(false); try { - Stat stat = zk.getZooKeeper().exists(getTXPath(fateId), false); + Stat stat = zk.exists(getTXPath(fateId), null); return stat.getCtime(); } catch (Exception e) { return 0; @@ -529,7 +532,7 @@ protected Optional getKey(FateId fateId) { private FateData getFateData(FateId fateId) { try { - return new FateData<>(zk.getData(getTXPath(fateId))); + return new FateData<>(zrw.getData(getTXPath(fateId))); } catch (NoNodeException nne) { return new FateData<>(TStatus.UNKNOWN, null, null, createEmptyRepoDeque(), createEmptyTxInfo()); @@ -540,7 +543,7 @@ private FateData getFateData(FateId fateId) { private FateData getFateData(FateId fateId, Stat stat) { try { - return new FateData<>(zk.getData(getTXPath(fateId), stat)); + return new FateData<>(zrw.getData(getTXPath(fateId), stat)); } catch (NoNodeException nne) { return new FateData<>(TStatus.UNKNOWN, null, null, createEmptyRepoDeque(), createEmptyTxInfo()); @@ -557,7 +560,7 @@ protected FateTxStore newUnreservedFateTxStore(FateId fateId) { @Override protected Stream getTransactions(EnumSet statuses) { try { - Stream stream = zk.getChildren(path).stream().map(strTxid -> { + Stream stream = zrw.getChildren(path).stream().map(strTxid -> { String txUUIDStr = strTxid.split("_")[1]; FateId fateId = FateId.from(fateInstanceType, txUUIDStr); // Memoizing for two reasons. First the status or reservation may never be requested, so @@ -612,7 +615,7 @@ private Map createEmptyTxInfo() { private byte[] mutate(FateId fateId, UnaryOperator> fateDataOp) throws KeeperException { try { - return zk.mutateExisting(getTXPath(fateId), currSerFateData -> { + return zrw.mutateExisting(getTXPath(fateId), currSerFateData -> { FateData currFateData = new FateData<>(currSerFateData); FateData newFateData = fateDataOp.apply(currFateData); if (newFateData == null) { 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 596037c1de1..af232424305 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.time.Duration; @@ -28,17 +29,18 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.LockSupport; +import java.util.function.Consumer; import java.util.function.Predicate; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; import org.apache.accumulo.core.util.cache.Caches; +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; @@ -52,10 +54,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 static final AtomicLong nextCacheId = new AtomicLong(0); private final String cacheId = "ZC" + nextCacheId.incrementAndGet(); @@ -65,7 +70,7 @@ public class ZooCache { // their compute functions. private final ConcurrentMap nodeCache; - private final ZooReader zReader; + private final ZooSession zk; private volatile boolean closed = false; @@ -101,19 +106,6 @@ public long getMzxid() { private final AtomicLong updateCount = new AtomicLong(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) { @@ -160,31 +152,39 @@ 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(), Duration.ofMinutes(3)); + } + /** * 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(reader, watcher, Duration.ofMinutes(3)); + public ZooCache(ZooSession zk, ZooCacheWatcher watcher) { + this(zk, Optional.of(watcher), Duration.ofMinutes(3)); } - public ZooCache(ZooReader reader, Watcher watcher, Duration timeout) { - this.zReader = reader; + public ZooCache(ZooSession zk, Optional watcher, Duration timeout) { + this.zk = requireNonNull(zk); this.externalWatcher = watcher; RemovalListener removalListerner = (path, zcNode, reason) -> { try { log.trace("{} removing watches for {} because {} accesses {}", cacheId, path, reason, zcNode == null ? -1 : zcNode.getAccessCount()); - reader.getZooKeeper().removeWatches(path, ZooCache.this.watcher, Watcher.WatcherType.Any, - false); + zk.removeWatches(path, ZooCache.this.watcher, Watcher.WatcherType.Any, false); } catch (InterruptedException | KeeperException | RuntimeException e) { log.warn("{} failed to remove watches on path {} in zookeeper", cacheId, path, e); } @@ -316,20 +316,18 @@ public List run() throws KeeperException, InterruptedException { if (zcn != null && zcn.cachedChildren()) { return zcn; } - try { - final ZooKeeper zooKeeper = getZooKeeper(); // Register a watcher on the node to monitor creation/deletion events for the node. It // is possible that an event from this watch could trigger prior to calling getChildren. // That is ok because the compute() call on the map has a lock and processing the event // will block until compute() returns. After compute() returns the event processing // would clear the map entry. - Stat stat = zooKeeper.exists(zPath, watcher); + Stat stat = zk.exists(zPath, watcher); if (stat == null) { log.trace("{} getChildren saw that {} does not exists", cacheId, zPath); return ZcNode.NON_EXISTENT; } - List children = zooKeeper.getChildren(zPath, watcher); + List children = zk.getChildren(zPath, watcher); log.trace("{} adding {} children of {} to cache", cacheId, children.size(), zPath); return new ZcNode(children, zcn); } catch (KeeperException.NoNodeException nne) { @@ -402,8 +400,7 @@ public byte[] run() throws KeeperException, InterruptedException { * non-existence can not be cached. */ try { - final ZooKeeper zooKeeper = getZooKeeper(); - Stat stat = zooKeeper.exists(zPath, watcher); + Stat stat = zk.exists(zPath, watcher); if (stat == null) { if (log.isTraceEnabled()) { log.trace("{} zookeeper did not contain {}", cacheId, zPath); @@ -413,7 +410,7 @@ public byte[] run() throws KeeperException, InterruptedException { byte[] data = null; ZcStat zstat = null; 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(e1); 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 b53fb597fbf..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; - } - - final 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 263f17e440c..a4934ae6eed 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.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.Objects; +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); } @@ -134,7 +140,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"); @@ -187,27 +193,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 1cc2a1b300d..3c7d72bd0fc 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 @@ -36,6 +36,7 @@ import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; import org.apache.accumulo.core.util.Timer; import org.apache.accumulo.core.util.UuidUtil; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; @@ -43,7 +44,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; @@ -87,7 +87,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; @@ -98,7 +98,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 { @@ -653,7 +653,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)); @@ -664,7 +664,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]; } @@ -787,7 +787,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/RootTabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java index 5d6f63a73b8..5eb075c809c 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/RootTabletMetadata.java @@ -71,7 +71,7 @@ public static String zooPath(ClientContext ctx) { public static RootTabletMetadata read(ClientContext ctx) { try { final String zpath = zooPath(ctx); - ZooReader zooReader = ctx.getZooReader(); + ZooReader zooReader = ctx.getZooSession().asReader(); // attempt (see ZOOKEEPER-1675) to ensure the latest root table metadata is read from // zookeeper zooReader.sync(zpath); 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/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 217303b2d9c..7f6f0e37c54 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 @@ -32,6 +32,7 @@ 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.apache.accumulo.core.lock.ServiceLockPaths; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,7 +59,8 @@ public void tearDown() { @Test public void testInvalidateCache() { - var zklc = new ZookeeperLockChecker(context); + var zklc = + new ZookeeperLockChecker(zc, context.getZooKeeperRoot(), new ServiceLockPaths(context)); verify(zc); reset(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 3aced2e103e..6d091d3aae1 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 @@ -249,7 +242,7 @@ public void testGetChildren_NoNode() throws Exception { assertTrue(zc.dataCached(ZPATH)); } - private static class TestWatcher implements Watcher { + private static class TestWatcher implements ZooCacheWatcher { private final WatchedEvent expectedEvent; private boolean wasCalled; @@ -259,7 +252,7 @@ private static class TestWatcher implements Watcher { } @Override - public void process(WatchedEvent event) { + public void accept(WatchedEvent event) { assertSame(expectedEvent, event); wasCalled = true; } @@ -294,7 +287,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); @@ -408,7 +401,7 @@ private void testGetBoth(boolean getDataFirst) 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)); @@ -442,7 +435,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/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 e8a55081c31..61a51e03705 100644 --- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java @@ -80,7 +80,6 @@ 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.core.fate.zookeeper.ZooSession; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.lock.ServiceLock.AccumuloLockWatcher; @@ -100,6 +99,7 @@ import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.compaction.CompactionPlannerInitParams; import org.apache.accumulo.core.util.compaction.CompactionServicesConfig; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.manager.state.SetGoalState; import org.apache.accumulo.minicluster.MiniAccumuloCluster; import org.apache.accumulo.minicluster.ServerType; @@ -121,7 +121,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,9 +153,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster { private final MiniAccumuloClusterControl clusterControl; private boolean initialized = false; - private ExecutorService executor; + private volatile ExecutorService executor; private ServiceLock miniLock; - private ZooKeeper zk; + private ZooSession miniLockZk; private AccumuloClient client; /** @@ -524,7 +523,7 @@ public synchronized void start() throws IOException, InterruptedException { InstanceId instanceIdFromFile = VolumeManager.getInstanceIDFromHdfs(instanceIdPath, hadoopConf); - ZooReaderWriter zrw = getServerContext().getZooReaderWriter(); + ZooReaderWriter zrw = getServerContext().getZooSession().asReaderWriter(); String instanceName = null; try { @@ -708,8 +707,8 @@ public void failedToAcquireLock(Exception e) { final int timeout = (int) ConfigurationTypeHelper.getTimeInMillis(properties.getOrDefault( Property.INSTANCE_ZK_TIMEOUT.getKey(), Property.INSTANCE_ZK_TIMEOUT.getDefaultValue())); final String secret = properties.get(Property.INSTANCE_SECRET.getKey()); - final byte[] auth = ("accumulo:" + secret).getBytes(UTF_8); - zk = ZooSession.getAuthenticatedSession(config.getZooKeepers(), timeout, "digest", auth); + miniLockZk = new ZooSession(MiniAccumuloClusterImpl.class.getSimpleName() + ".lock", + config.getZooKeepers(), timeout, secret); // It's possible start was called twice... if (miniLock == null) { @@ -722,12 +721,12 @@ public void failedToAcquireLock(Exception e) { String miniZDirPath = miniZInstancePath.substring(0, miniZInstancePath.indexOf("/" + miniUUID.toString())); try { - if (zk.exists(miniZDirPath, null) == null) { - zk.create(miniZDirPath, new byte[0], ZooUtil.PUBLIC, CreateMode.PERSISTENT); + if (miniLockZk.exists(miniZDirPath, null) == null) { + miniLockZk.create(miniZDirPath, new byte[0], ZooUtil.PUBLIC, CreateMode.PERSISTENT); log.info("Created: {}", miniZDirPath); } - if (zk.exists(miniZInstancePath, null) == null) { - zk.create(miniZInstancePath, new byte[0], ZooUtil.PUBLIC, CreateMode.PERSISTENT); + if (miniLockZk.exists(miniZInstancePath, null) == null) { + miniLockZk.create(miniZInstancePath, new byte[0], ZooUtil.PUBLIC, CreateMode.PERSISTENT); log.info("Created: {}", miniZInstancePath); } } catch (KeeperException | InterruptedException e) { @@ -735,7 +734,7 @@ public void failedToAcquireLock(Exception e) { } ServiceLockData sld = new ServiceLockData(miniUUID, "localhost", ThriftService.NONE, Constants.DEFAULT_RESOURCE_GROUP_NAME); - miniLock = new ServiceLock(zk, slp, miniUUID); + miniLock = new ServiceLock(miniLockZk, slp, miniUUID); miniLock.lock(miniLockWatcher, sld); lockWatcherInvoked.await(); @@ -981,9 +980,9 @@ public synchronized void stop() throws IOException, InterruptedException { miniLock = null; this.getServerContext().clearServiceLock(); } - if (zk != null) { - zk.close(); - zk = null; + if (miniLockZk != null) { + miniLockZk.close(); + miniLockZk = null; } if (client != null) { client.close(); 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 fc0dd5f3736..acb6c80c311 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 @@ -43,6 +43,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; @@ -51,8 +52,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.lock.ServiceLock; import org.apache.accumulo.core.metadata.schema.Ample; @@ -94,7 +93,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; @@ -112,16 +110,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)); @@ -144,21 +144,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() { @@ -214,15 +217,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 */ @@ -320,7 +314,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/compaction/CompactionConfigStorage.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionConfigStorage.java index 0b0bedb645c..970a4f1c9bf 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionConfigStorage.java +++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionConfigStorage.java @@ -69,7 +69,7 @@ public static CompactionConfig getConfig(ServerContext context, FateId fateId) public static CompactionConfig getConfig(ServerContext context, FateId fateId, Predicate tableIdPredicate) throws InterruptedException, KeeperException { try { - byte[] data = context.getZooReaderWriter().getData(createPath(context, fateId)); + byte[] data = context.getZooSession().asReader().getData(createPath(context, fateId)); try (ByteArrayInputStream bais = new ByteArrayInputStream(data); DataInputStream dis = new DataInputStream(bais)) { var tableId = TableId.of(dis.readUTF()); @@ -89,13 +89,13 @@ public static CompactionConfig getConfig(ServerContext context, FateId fateId, public static void setConfig(ServerContext context, FateId fateId, byte[] encConfig) throws InterruptedException, KeeperException { - context.getZooReaderWriter().putPrivatePersistentData(createPath(context, fateId), encConfig, - ZooUtil.NodeExistsPolicy.SKIP); + context.getZooSession().asReaderWriter().putPrivatePersistentData(createPath(context, fateId), + encConfig, ZooUtil.NodeExistsPolicy.SKIP); } public static void deleteConfig(ServerContext context, FateId fateId) throws InterruptedException, KeeperException { - context.getZooReaderWriter().delete(createPath(context, fateId)); + context.getZooSession().asReaderWriter().delete(createPath(context, fateId)); } public static Map getAllConfig(ServerContext context, @@ -103,7 +103,7 @@ public static Map getAllConfig(ServerContext context, Map configs = new HashMap<>(); - var children = context.getZooReaderWriter() + var children = context.getZooSession().asReader() .getChildren(context.getZooKeeperRoot() + Constants.ZCOMPACTIONS); for (var child : children) { String[] fields = child.split(DELIMITER); 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 c3a5553de97..3089189f94e 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 @@ -97,7 +97,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 b6e51b412f7..f20aa6dfb55 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 @@ -30,6 +30,7 @@ import java.util.Optional; 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.ThriftSecurityException; @@ -37,6 +38,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.lock.ServiceLockPaths; @@ -60,14 +62,14 @@ 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.annotations.VisibleForTesting; +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); @@ -77,7 +79,6 @@ public interface Listener { private final Listener cback; private final ServerContext context; - private ZooCache zooCache; public class TServerConnection { private final HostAndPort address; @@ -208,16 +209,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() { @@ -249,7 +250,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 @@ -304,7 +305,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 @@ -475,7 +476,7 @@ public synchronized void remove(TServerInstance server) { ServiceLockPath slp = paths.iterator().next(); log.info("Removing zookeeper lock for {}", slp); try { - context.getZooReaderWriter().recursiveDelete(slp.toString(), SKIP); + context.getZooSession().asReaderWriter().recursiveDelete(slp.toString(), 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 966167d527f..d12f4e9363a 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 @@ -47,18 +47,15 @@ public class DeadServerList { // and replace the "UNKNOWN" value with the ResourceGroup private static final String RESOURCE_GROUP = "UNKNOWN"; private final ServerContext ctx; - private final String root; private final ZooReaderWriter zoo; private final String path; public DeadServerList(ServerContext context) { this.ctx = context; - zoo = this.ctx.getZooReaderWriter(); - root = this.ctx.getZooKeeperRoot(); - - this.path = root + Constants.ZDEADTSERVERS + "/" + RESOURCE_GROUP; + this.zoo = context.getZooSession().asReaderWriter(); + this.path = context.getZooKeeperRoot() + Constants.ZDEADTSERVERS + "/" + RESOURCE_GROUP; try { - ctx.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/RootConditionalWriter.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java index 0e28d1a7a71..9f61097f8e9 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootConditionalWriter.java @@ -97,7 +97,7 @@ public Result write(ConditionalMutation mutation) { List results = new ArrayList<>(); try { - context.getZooReaderWriter().mutateExisting(zpath, currVal -> { + context.getZooSession().asReaderWriter().mutateExisting(zpath, currVal -> { String currJson = new String(currVal, UTF_8); var rtm = new RootTabletMetadata(currJson); 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 6bf8cbca260..3f429b3c715 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 @@ -106,7 +106,7 @@ public void mutate() { // TODO examine implementation of getZooReaderWriter().mutate() // TODO for efficiency this should maybe call mutateExisting - 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 dcea93dea97..59671a0070c 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 @@ -106,7 +106,7 @@ private void mutateRootGcCandidates(Consumer mutator) { String zpath = context.getZooKeeperRoot() + ZROOT_TABLET_GC_CANDIDATES; try { // TODO calling create seems unnecessary and is possibly racy and inefficient - 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); @@ -231,7 +231,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 493d4570a10..da12470b87b 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.TFateOperation; 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 86e33cc8f2a..3bd0f091deb 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); @@ -113,7 +113,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); } @@ -121,8 +121,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(); } @@ -249,9 +249,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 d5a5ca4e55f..c1d2bf71081 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 @@ -73,7 +73,6 @@ import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; 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.lock.ServiceLockPaths.AddressSelector; import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; @@ -890,7 +889,7 @@ private void executeFateOpsCommand(ServerContext context, FateOpsCommand fateOps var zLockManagerPath = context.getServerPaths().createManagerPath(); var zTableLocksPath = context.getServerPaths().createTableLocksPath(); String fateZkPath = zkRoot + Constants.ZFATE; - ZooReaderWriter zk = context.getZooReaderWriter(); + var zk = context.getZooSession(); MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk, createDummyLockID(), null); UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); Map> fateStores = @@ -978,7 +977,7 @@ private void summarizeFateTx(ServerContext context, FateOpsCommand cmd, AdminUti Map> fateStores, ServiceLockPath tableLocksPath) throws InterruptedException, AccumuloException, AccumuloSecurityException, KeeperException { - ZooReaderWriter zk = context.getZooReaderWriter(); + var zk = context.getZooSession(); var transactions = admin.getStatus(fateStores, zk, tableLocksPath, null, 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 af98fd0d201..e65a5e02a61 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 @@ -40,6 +40,7 @@ import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; import org.apache.accumulo.core.lock.ServiceLockPaths; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +69,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) { @@ -81,43 +82,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 f950975c802..69782c4d35e 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 @@ -57,19 +57,18 @@ public ServiceStatusCmd() {} */ public void execute(final ServerContext context, final Opts opts) { - ZooReader zooReader = context.getZooReader(); - - final String zooRoot = context.getZooKeeperRoot(); - LOG.trace("zooRoot: {}", zooRoot); + if (LOG.isTraceEnabled()) { + LOG.trace("zooRoot: {}", context.getZooKeeperRoot()); + } final Map services = new TreeMap<>(); - services.put(ServiceStatusReport.ReportKey.MANAGER, getManagerStatus(zooReader, context)); - services.put(ServiceStatusReport.ReportKey.MONITOR, getMonitorStatus(zooReader, context)); + services.put(ServiceStatusReport.ReportKey.MANAGER, getManagerStatus(context)); + services.put(ServiceStatusReport.ReportKey.MONITOR, getMonitorStatus(context)); services.put(ServiceStatusReport.ReportKey.T_SERVER, getTServerStatus(context)); services.put(ServiceStatusReport.ReportKey.S_SERVER, getScanServerStatus(context)); services.put(ServiceStatusReport.ReportKey.COMPACTOR, getCompactorStatus(context)); - services.put(ServiceStatusReport.ReportKey.GC, getGcStatus(zooReader, context)); + services.put(ServiceStatusReport.ReportKey.GC, getGcStatus(context)); ServiceStatusReport report = new ServiceStatusReport(services, opts.noHosts); @@ -87,9 +86,9 @@ public void execute(final ServerContext context, final Opts opts) { * lock data providing a service descriptor with host and port. */ @VisibleForTesting - StatusSummary getManagerStatus(ZooReader zooReader, ServerContext context) { + StatusSummary getManagerStatus(ServerContext context) { String lockPath = context.getServerPaths().createManagerPath().toString(); - return getStatusSummary(ServiceStatusReport.ReportKey.MANAGER, zooReader, lockPath); + return getStatusSummary(ServiceStatusReport.ReportKey.MANAGER, context, lockPath); } /** @@ -97,9 +96,9 @@ StatusSummary getManagerStatus(ZooReader zooReader, ServerContext context) { * lock data providing a service descriptor with host and port. */ @VisibleForTesting - StatusSummary getMonitorStatus(final ZooReader zooReader, ServerContext context) { + StatusSummary getMonitorStatus(ServerContext context) { String lockPath = context.getServerPaths().createMonitorPath().toString(); - return getStatusSummary(ServiceStatusReport.ReportKey.MONITOR, zooReader, lockPath); + return getStatusSummary(ServiceStatusReport.ReportKey.MONITOR, context, lockPath); } /** @@ -141,9 +140,9 @@ StatusSummary getScanServerStatus(ServerContext context) { * providing GC_CLIENT=host:port */ @VisibleForTesting - StatusSummary getGcStatus(final ZooReader zooReader, ServerContext context) { + StatusSummary getGcStatus(ServerContext context) { String lockPath = context.getServerPaths().createGarbageCollectorPath().toString(); - return getStatusSummary(ServiceStatusReport.ReportKey.GC, zooReader, lockPath); + return getStatusSummary(ServiceStatusReport.ReportKey.GC, context, lockPath); } /** @@ -170,8 +169,8 @@ StatusSummary getCompactorStatus(ServerContext context) { * @return service status */ private StatusSummary getStatusSummary(ServiceStatusReport.ReportKey displayNames, - ZooReader zooReader, String lockPath) { - var result = readAllNodesData(zooReader, lockPath); + ServerContext context, String lockPath) { + var result = readAllNodesData(context.getZooSession().asReader(), lockPath); Map> byGroup = new TreeMap<>(); result.getData().forEach(data -> { ServiceLockData.ServiceDescriptors sld = ServiceLockData.parseServiceDescriptors(data); 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 369be4bee33..f075f50cf9a 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 @@ -39,7 +39,7 @@ public static void execute(final ServerContext context, final String lock, final throws Exception { ZooCache cache = context.getZooCache(); - ZooReaderWriter zoo = context.getZooReaderWriter(); + ZooReaderWriter zoo = context.getZooSession().asReaderWriter(); if (delete == null) { Set tabletServers = 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 304af5d5ef7..4f98128fe1c 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 @@ -102,11 +102,11 @@ public void execute(String[] args) throws Exception { } try (var context = new ServerContext(siteConf)) { - final ZooReaderWriter zoo = context.getZooReaderWriter(); + final ZooReaderWriter zrw = context.getZooSession().asReaderWriter(); if (opts.zapManager) { ServiceLockPath managerLockPath = context.getServerPaths().createManagerPath(); try { - zapDirectory(zoo, managerLockPath, opts); + zapDirectory(zrw, managerLockPath, opts); } catch (KeeperException | InterruptedException e) { e.printStackTrace(); } @@ -128,10 +128,10 @@ public void execute(String[] args) throws Exception { message("Deleting " + tserverPath + " from zookeeper", opts); if (opts.zapManager) { - zoo.recursiveDelete(tserverPath.toString(), NodeMissingPolicy.SKIP); + zrw.recursiveDelete(tserverPath.toString(), NodeMissingPolicy.SKIP); } else { - if (!zoo.getChildren(tserverPath.toString()).isEmpty()) { - if (!ServiceLock.deleteLock(zoo, tserverPath, "tserver")) { + if (!zrw.getChildren(tserverPath.toString()).isEmpty()) { + if (!ServiceLock.deleteLock(zrw, tserverPath, "tserver")) { message("Did not delete " + tserverPath, opts); } } @@ -151,7 +151,7 @@ public void execute(String[] args) throws Exception { try { for (String group : compactorResourceGroupPaths) { message("Deleting " + group + " from zookeeper", opts); - zoo.recursiveDelete(group, NodeMissingPolicy.SKIP); + zrw.recursiveDelete(group, NodeMissingPolicy.SKIP); } } catch (KeeperException | InterruptedException e) { log.error("Error deleting compactors from zookeeper, {}", e.getMessage(), e); @@ -165,8 +165,8 @@ public void execute(String[] args) throws Exception { context.getServerPaths().getScanServer(rgp, AddressSelector.all(), false); for (ServiceLockPath sserverPath : sserverLockPaths) { message("Deleting " + sserverPath + " from zookeeper", opts); - if (!zoo.getChildren(sserverPath.toString()).isEmpty()) { - ServiceLock.deleteLock(zoo, sserverPath); + if (!zrw.getChildren(sserverPath.toString()).isEmpty()) { + ServiceLock.deleteLock(zrw, sserverPath); } } } catch (KeeperException | InterruptedException e) { 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 99dee426ca4..9ebc3189124 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.getKeyValues().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 c7c04079c56..4a1438dc379 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 @@ -30,7 +30,6 @@ import org.apache.accumulo.core.fate.FateInstanceType; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.cli.ServerUtilOpts; @@ -65,7 +64,7 @@ private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext cont final String zkRoot = context.getZooKeeperRoot(); final var zTableLocksPath = context.getServerPaths().createTableLocksPath(); final String fateZkPath = zkRoot + Constants.ZFATE; - final ZooReaderWriter zk = context.getZooReaderWriter(); + final var zk = context.getZooSession(); final MetaFateStore mfs = new MetaFateStore<>(fateZkPath, zk, AbstractFateStore.createDummyLockID(), null); final UserFateStore ufs = new UserFateStore<>(context, AccumuloTable.FATE.tableName(), @@ -75,7 +74,8 @@ private static Admin.CheckCommand.CheckStatus checkTableLocks(ServerContext cont 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 73c9796ab32..fb374ce2293 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 @@ -188,7 +188,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/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 6ffdfb5437d..99c4b04bacf 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(); @@ -81,7 +85,7 @@ public void initCommonMocks() throws Exception { @AfterEach public void verifyMocks() { - verify(context, zrw, readyMonitor); + verify(context, zk, zrw, readyMonitor); } @Test @@ -97,9 +101,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); @@ -127,9 +131,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(); @@ -159,9 +163,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(); @@ -193,9 +197,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(); @@ -217,9 +221,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(); @@ -254,9 +258,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 aee14ba4d4b..dc61b8c91e2 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 344e09a5388..41c5779adaf 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 @@ -23,6 +23,7 @@ import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.isA; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -42,11 +43,11 @@ import org.apache.accumulo.core.fate.zookeeper.ZooReader; import org.apache.accumulo.core.fate.zookeeper.ZooUtil; import org.apache.accumulo.core.lock.ServiceLockPaths; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.util.serviceStatus.ServiceStatusReport; import org.apache.accumulo.server.util.serviceStatus.StatusSummary; import org.apache.zookeeper.KeeperException; -import org.easymock.EasyMock; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -57,36 +58,33 @@ public class ServiceStatusCmdTest { private static final Logger LOG = LoggerFactory.getLogger(ServiceStatusCmdTest.class); - private ServerContext context; private String zRoot; - private ZooReader zooReader; + private ServerContext context; + private ZooSession zooReader; private ZooCache zooCache; @BeforeEach public void populateContext() { - InstanceId iid = InstanceId.of(UUID.randomUUID()); - zRoot = ZooUtil.getRoot(iid); + zRoot = ZooUtil.getRoot(InstanceId.of(UUID.randomUUID())); context = createMock(ServerContext.class); - expect(context.getInstanceID()).andReturn(iid).anyTimes(); - expect(context.getZooKeeperRoot()).andReturn(zRoot).anyTimes(); - - zooReader = createMock(ZooReader.class); + zooReader = createMock(ZooSession.class); zooCache = createMock(ZooCache.class); - - expect(context.getZooReader()).andReturn(zooReader).anyTimes(); + expect(zooReader.asReader()).andReturn(new ZooReader(zooReader)).anyTimes(); expect(context.getZooCache()).andReturn(zooCache).anyTimes(); + expect(context.getZooSession()).andReturn(zooReader).anyTimes(); + expect(context.getZooKeeperRoot()).andReturn(zRoot).anyTimes(); expect(context.getServerPaths()).andReturn(new ServiceLockPaths(context)).anyTimes(); - replay(context); } @AfterEach public void validateMocks() { - verify(context, zooReader); + verify(context, zooReader, zooCache); } @Test - void testManagerHosts() throws Exception { + public void testManagerHosts() throws Exception { + replay(zooCache); String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; @@ -99,19 +97,18 @@ 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)) - .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock1Name))).andReturn(lock1data.getBytes(UTF_8)) - .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lock2Data.getBytes(UTF_8)) - .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock3Name))).andReturn(lock3Data.getBytes(UTF_8)) - .anyTimes(); - - replay(zooCache, zooReader); + expect(zooReader.getChildren(lockPath, null)) + .andReturn(List.of(lock1Name, lock2Name, lock3Name)); + expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) + .andReturn(lock1data.getBytes(UTF_8)); + expect(zooReader.getData(lockPath + "/" + lock2Name, null, null)) + .andReturn(lock2Data.getBytes(UTF_8)); + expect(zooReader.getData(lockPath + "/" + lock3Name, null, null)) + .andReturn(lock3Data.getBytes(UTF_8)); + replay(zooReader); ServiceStatusCmd cmd = new ServiceStatusCmd(); - StatusSummary status = cmd.getManagerStatus(zooReader, context); + StatusSummary status = cmd.getManagerStatus(context); LOG.info("manager status data: {}", status); assertEquals(3, status.getServiceCount()); @@ -134,7 +131,8 @@ void testManagerHosts() throws Exception { } @Test - void testMonitorHosts() throws Exception { + public void testMonitorHosts() throws Exception { + replay(zooCache); String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; @@ -144,16 +142,16 @@ 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)) - .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(host2.getBytes(UTF_8)) - .anyTimes(); + expect(zooReader.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)); + expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) + .andReturn(host1.getBytes(UTF_8)); + expect(zooReader.getData(lockPath + "/" + lock2Name, null, null)) + .andReturn(host2.getBytes(UTF_8)); replay(zooReader); ServiceStatusCmd cmd = new ServiceStatusCmd(); - StatusSummary status = cmd.getMonitorStatus(zooReader, context); + StatusSummary status = cmd.getMonitorStatus(context); LOG.info("monitor status data: {}", status); assertEquals(2, status.getServiceCount()); @@ -175,7 +173,8 @@ void testMonitorHosts() throws Exception { } @Test - void testTServerHosts() throws Exception { + public void testTServerHosts() throws Exception { + replay(zooReader); String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; @@ -219,38 +218,33 @@ void testTServerHosts() throws Exception { + host3 + "\",\"group\":\"default\"}]}"; String basePath = zRoot + Constants.ZTSERVERS; - - expect(zooCache.getChildren(zRoot)).andReturn(List.of(Constants.ZTSERVERS)).anyTimes(); - expect(zooCache.getChildren(basePath)).andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME)) - .anyTimes(); + expect(zooCache.getChildren(basePath)) + .andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME)); expect(zooCache.getChildren(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME)) - .andReturn(List.of(host1, host2, host3)).anyTimes(); + .andReturn(List.of(host1, host2, host3)); expect( zooCache.getChildren(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host1)) - .andReturn(List.of(lock1Name)).anyTimes(); + .andReturn(List.of(lock1Name)); expect(zooCache.get( - EasyMock.eq( - basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host1 + "/" + lock1Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData1.getBytes(UTF_8)).anyTimes(); + eq(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host1 + "/" + lock1Name), + isA(ZcStat.class))).andReturn(lockData1.getBytes(UTF_8)); expect( zooCache.getChildren(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2)) - .andReturn(List.of(lock2Name)).anyTimes(); + .andReturn(List.of(lock2Name)); expect(zooCache.get( - EasyMock.eq( - basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2 + "/" + lock2Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData2.getBytes(UTF_8)).anyTimes(); + eq(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2 + "/" + lock2Name), + isA(ZcStat.class))).andReturn(lockData2.getBytes(UTF_8)); expect( zooCache.getChildren(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host3)) - .andReturn(List.of(lock3Name)).anyTimes(); + .andReturn(List.of(lock3Name)); expect(zooCache.get( - EasyMock.eq( - basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host3 + "/" + lock3Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData3.getBytes(UTF_8)).anyTimes(); + eq(basePath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host3 + "/" + lock3Name), + isA(ZcStat.class))).andReturn(lockData3.getBytes(UTF_8)); - replay(zooCache, zooReader); + replay(zooCache); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getTServerStatus(context); @@ -275,12 +269,11 @@ void testTServerHosts() throws Exception { assertEquals(expected.getServiceCount(), status.getServiceCount()); assertEquals(expected.getErrorCount(), status.getErrorCount()); assertEquals(expected, status); - - verify(zooCache); } @Test - void testScanServerHosts() throws Exception { + public void testScanServerHosts() throws Exception { + replay(zooReader); UUID uuid1 = UUID.randomUUID(); String lock1Name = "zlock#" + uuid1 + "#0000000001"; UUID uuid2 = UUID.randomUUID(); @@ -316,40 +309,35 @@ void testScanServerHosts() throws Exception { + host4 + "\",\"group\":\"default\"}]}"; String lockPath = zRoot + Constants.ZSSERVERS; - expect(zooCache.getChildren(zRoot)).andReturn(List.of(Constants.ZSSERVERS)).anyTimes(); expect(zooCache.getChildren(lockPath)) - .andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME, "sg1")).anyTimes(); + .andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME, "sg1")); expect(zooCache.getChildren(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME)) - .andReturn(List.of(host2, host4)).anyTimes(); - expect(zooCache.getChildren(lockPath + "/sg1")).andReturn(List.of(host1, host3)).anyTimes(); + .andReturn(List.of(host2, host4)); + expect(zooCache.getChildren(lockPath + "/sg1")).andReturn(List.of(host1, host3)); - expect(zooCache.getChildren(lockPath + "/sg1/" + host1)).andReturn(List.of(lock1Name)) - .anyTimes(); - expect(zooCache.get(EasyMock.eq(lockPath + "/sg1/" + host1 + "/" + lock1Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData1.getBytes(UTF_8)).anyTimes(); + expect(zooCache.getChildren(lockPath + "/sg1/" + host1)).andReturn(List.of(lock1Name)); + expect(zooCache.get(eq(lockPath + "/sg1/" + host1 + "/" + lock1Name), isA(ZcStat.class))) + .andReturn(lockData1.getBytes(UTF_8)); expect( zooCache.getChildren(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2)) - .andReturn(List.of(lock2Name)).anyTimes(); + .andReturn(List.of(lock2Name)); expect(zooCache.get( - EasyMock.eq( - lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2 + "/" + lock2Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData2.getBytes(UTF_8)).anyTimes(); + eq(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host2 + "/" + lock2Name), + isA(ZcStat.class))).andReturn(lockData2.getBytes(UTF_8)); - expect(zooCache.getChildren(lockPath + "/sg1/" + host3)).andReturn(List.of(lock3Name)) - .anyTimes(); - expect(zooCache.get(EasyMock.eq(lockPath + "/sg1/" + host3 + "/" + lock3Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData3.getBytes(UTF_8)).anyTimes(); + expect(zooCache.getChildren(lockPath + "/sg1/" + host3)).andReturn(List.of(lock3Name)); + expect(zooCache.get(eq(lockPath + "/sg1/" + host3 + "/" + lock3Name), isA(ZcStat.class))) + .andReturn(lockData3.getBytes(UTF_8)); expect( zooCache.getChildren(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host4)) - .andReturn(List.of(lock4Name)).anyTimes(); + .andReturn(List.of(lock4Name)); expect(zooCache.get( - EasyMock.eq( - lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host4 + "/" + lock4Name), - EasyMock.isA(ZcStat.class))).andReturn(lockData4.getBytes(UTF_8)).anyTimes(); + eq(lockPath + "/" + Constants.DEFAULT_RESOURCE_GROUP_NAME + "/" + host4 + "/" + lock4Name), + isA(ZcStat.class))).andReturn(lockData4.getBytes(UTF_8)); - replay(zooCache, zooReader); + replay(zooCache); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getScanServerStatus(context); @@ -363,37 +351,32 @@ void testScanServerHosts() throws Exception { Set.of("default", "sg1"), hostByGroup, 0); assertEquals(expected, status); - verify(zooCache); } @Test public void testCompactorStatus() throws Exception { + replay(zooReader); String lockPath = zRoot + Constants.ZCOMPACTORS; + expect(zooCache.getChildren(lockPath)).andReturn(List.of("q1", "q2")); + expect(zooCache.getChildren(lockPath + "/q1")).andReturn(List.of("hostA:8080", "hostC:8081")); + expect(zooCache.getChildren(lockPath + "/q2")).andReturn(List.of("hostB:9090", "hostD:9091")); + expect(zooCache.getChildren(lockPath + "/q1/hostA:8080")).andReturn(List.of()); + expect(zooCache.getChildren(lockPath + "/q1/hostC:8081")).andReturn(List.of()); + expect(zooCache.getChildren(lockPath + "/q2/hostB:9090")).andReturn(List.of()); + expect(zooCache.getChildren(lockPath + "/q2/hostD:9091")).andReturn(List.of()); - expect(zooCache.getChildren(zRoot)).andReturn(List.of(Constants.ZCOMPACTORS)).anyTimes(); - expect(zooCache.getChildren(lockPath)).andReturn(List.of("q1", "q2")).anyTimes(); - expect(zooCache.getChildren(lockPath + "/q1")).andReturn(List.of("hostA:8080", "hostC:8081")) - .anyTimes(); - expect(zooCache.getChildren(lockPath + "/q2")).andReturn(List.of("hostB:9090", "hostD:9091")) - .anyTimes(); - expect(zooCache.getChildren(lockPath + "/q1/hostA:8080")).andReturn(List.of()).anyTimes(); - expect(zooCache.getChildren(lockPath + "/q1/hostC:8081")).andReturn(List.of()).anyTimes(); - expect(zooCache.getChildren(lockPath + "/q2/hostB:9090")).andReturn(List.of()).anyTimes(); - expect(zooCache.getChildren(lockPath + "/q2/hostD:9091")).andReturn(List.of()).anyTimes(); - - replay(zooCache, zooReader); + replay(zooCache); ServiceStatusCmd cmd = new ServiceStatusCmd(); StatusSummary status = cmd.getCompactorStatus(context); LOG.info("compactor group counts: {}", status); assertEquals(0, status.getResourceGroups().size()); - - verify(zooCache); } @Test public void testGcHosts() throws Exception { + replay(zooCache); String lockPath = zRoot + ZGC_LOCK; UUID uuid1 = UUID.randomUUID(); @@ -411,16 +394,16 @@ 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)) - .once(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lockData2.getBytes(UTF_8)) - .once(); + expect(zooReader.getChildren(lockPath, null)).andReturn(List.of(lock1Name, lock2Name)); + expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) + .andReturn(lockData1.getBytes(UTF_8)); + expect(zooReader.getData(lockPath + "/" + lock2Name, null, null)) + .andReturn(lockData2.getBytes(UTF_8)); replay(zooReader); ServiceStatusCmd cmd = new ServiceStatusCmd(); - StatusSummary status = cmd.getGcStatus(zooReader, context); + StatusSummary status = cmd.getGcStatus(context); LOG.info("gc server counts: {}", status); assertEquals(2, status.getResourceGroups().size()); assertEquals(2, status.getServiceCount()); @@ -438,7 +421,8 @@ public void testGcHosts() throws Exception { * error is skipped and available hosts are returned. */ @Test - void zkNodeDeletedTest() throws Exception { + public void zkNodeDeletedTest() throws Exception { + replay(zooCache); String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001"; String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002"; String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003"; @@ -454,21 +438,19 @@ void zkNodeDeletedTest() throws Exception { + host3 + "\",\"group\":\"manager1\"}]}"; String lockPath = zRoot + Constants.ZMANAGER_LOCK; - expect(zooReader.getChildren(eq(lockPath))).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))) - .andThrow(new KeeperException.NoNodeException("no node forced exception")).once(); - expect(zooReader.getData(eq(lockPath + "/" + lock2Name))).andReturn(lock2Data.getBytes(UTF_8)) - .anyTimes(); - expect(zooReader.getData(eq(lockPath + "/" + lock3Name))).andReturn(lock3Data.getBytes(UTF_8)) - .anyTimes(); + expect(zooReader.getChildren(lockPath, null)) + .andReturn(List.of(lock1Name, lock2Name, lock3Name)); + expect(zooReader.getData(lockPath + "/" + lock1Name, null, null)) + .andThrow(new KeeperException.NoNodeException("no node forced exception")); + expect(zooReader.getData(lockPath + "/" + lock2Name, null, null)) + .andReturn(lock2Data.getBytes(UTF_8)); + expect(zooReader.getData(lockPath + "/" + lock3Name, null, null)) + .andReturn(lock3Data.getBytes(UTF_8)); replay(zooReader); ServiceStatusCmd cmd = new ServiceStatusCmd(); - StatusSummary status = cmd.getManagerStatus(zooReader, context); + StatusSummary status = cmd.getManagerStatus(context); LOG.info("manager status data: {}", status); assertEquals(2, status.getServiceByGroups().size()); @@ -483,7 +465,7 @@ void zkNodeDeletedTest() throws Exception { @Test public void testServiceStatusCommandOpts() { - replay(zooReader); // needed for @AfterAll verify + replay(zooReader, zooCache); ServiceStatusCmd.Opts opts = new ServiceStatusCmd.Opts(); assertFalse(opts.json); assertFalse(opts.noHosts); 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 214c3af561b..20502cfcaa0 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 @@ -272,7 +272,7 @@ protected void checkIfCanceled() { protected void announceExistence(HostAndPort clientAddress) throws KeeperException, InterruptedException { - ZooReaderWriter zoo = getContext().getZooReaderWriter(); + ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); final ServiceLockPath path = getContext().getServerPaths().createCompactorPath(getResourceGroup(), clientAddress); @@ -284,16 +284,13 @@ protected void announceExistence(HostAndPort clientAddress) try { zoo.mkdirs(compactorGroupPath); zoo.putPersistentData(path.toString(), 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."); - } + } catch (KeeperException.NoAuthException e) { + LOG.error("Failed to write to ZooKeeper. Ensure that" + + " accumulo.properties, specifically instance.secret, is consistent."); throw e; } - compactorLock = - new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, compactorId); + compactorLock = new ServiceLock(getContext().getZooSession(), path, 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 e8f3dcb3db7..efef0dd72ca 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 @@ -369,7 +369,7 @@ private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedExc var path = getContext().getServerPaths().createGarbageCollectorPath(); 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 936319f880b..17ae1d10d74 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 @@ -486,7 +486,7 @@ protected Manager(ConfigOpts opts, Function ser 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); @@ -505,7 +505,7 @@ public TServerConnection getConnection(TServerInstance server) { 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) { @@ -516,7 +516,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) { @@ -633,7 +633,7 @@ private class ScanServerZKCleaner implements Runnable { @Override public void run() { - final ZooReaderWriter zrw = getContext().getZooReaderWriter(); + final ZooReaderWriter zrw = getContext().getZooSession().asReaderWriter(); while (stillManager()) { try { @@ -1195,7 +1195,7 @@ public void run() { // Don't call start the CompactionCoordinator until we have tservers. compactionCoordinator.start(); - ZooReaderWriter zReaderWriter = context.getZooReaderWriter(); + ZooReaderWriter zReaderWriter = context.getZooSession().asReaderWriter(); try { zReaderWriter.getChildren(zroot + Constants.ZRECOVERY, new Watcher() { @@ -1267,8 +1267,8 @@ boolean canSuspendTablets() { Predicate isLockHeld = lock -> ServiceLock.isLockHeld(context.getZooCache(), lock); var metaInstance = initializeFateInstance(context, - new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, - context.getZooReaderWriter(), managerLock.getLockID(), isLockHeld)); + new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, context.getZooSession(), + managerLock.getLockID(), isLockHeld)); var userInstance = initializeFateInstance(context, new UserFateStore<>(context, AccumuloTable.FATE.tableName(), managerLock.getLockID(), isLockHeld)); @@ -1488,7 +1488,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 63137097a28..2bf22a28418 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 @@ -112,7 +112,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/compaction/coordinator/CompactionCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java index e3bec68c82e..4dc42c11379 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java @@ -1120,7 +1120,7 @@ private void deleteEmpty(ZooReaderWriter zoorw, String path) private void cleanUpEmptyCompactorPathInZK() { final String compactorQueuesPath = this.ctx.getZooKeeperRoot() + Constants.ZCOMPACTORS; - final var zoorw = this.ctx.getZooReaderWriter(); + final var zoorw = this.ctx.getZooSession().asReaderWriter(); final double queueSizeFactor = ctx.getConfiguration() .getFraction(Property.MANAGER_COMPACTION_SERVICE_PRIORITY_QUEUE_SIZE_FACTOR); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java index a0923d18642..812abba18da 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java @@ -81,7 +81,7 @@ public static MetaFateMetricValues getMetaStoreMetrics(final ServerContext conte try { builder = getFateMetrics(metaFateStore, new Builder()); - 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/meta/MetaFateMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java index d26cf259d9b..8412a17aad3 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetrics.java @@ -64,7 +64,7 @@ public void registerMetrics(MeterRegistry registry) { @Override protected ReadOnlyFateStore> buildStore(ServerContext context) { try { - return new MetaFateStore<>(getFateRootPath(context), context.getZooReaderWriter(), + return new MetaFateStore<>(getFateRootPath(context), context.getZooSession(), AbstractFateStore.createDummyLockID(), null); } catch (KeeperException ex) { throw new IllegalStateException( 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 7270d8018f4..7a4d5db9c37 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 @@ -76,7 +76,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/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 36622789443..20a962cbbe1 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 @@ -74,19 +74,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())) { @@ -127,7 +128,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); @@ -149,7 +150,7 @@ public static long reserveTable(Manager env, TableId tableId, FateId fateId, Loc boolean tableMustExist, TableOperation op) throws Exception { if (getLock(env.getContext(), tableId, fateId, 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"); @@ -178,7 +179,7 @@ public static long reserveNamespace(Manager env, NamespaceId namespaceId, FateId LockType lockType, boolean mustExist, TableOperation op) throws Exception { if (getLock(env.getContext(), namespaceId, fateId, 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, @@ -197,7 +198,7 @@ public static long reserveHdfsDirectory(Manager env, String directory, FateId fa 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, fateId, "")) { return 0; @@ -210,7 +211,7 @@ public static void unreserveHdfsDirectory(Manager env, String directory, FateId throws KeeperException, InterruptedException { String resvPath = env.getContext().getZooKeeperRoot() + Constants.ZHDFS_RESERVATIONS + "/" + Base64.getEncoder().encodeToString(directory.getBytes(UTF_8)); - ZooReservation.release(env.getContext().getZooReaderWriter(), resvPath, fateId); + ZooReservation.release(env.getContext().getZooSession().asReaderWriter(), resvPath, fateId); } private static Lock getLock(ServerContext context, AbstractId id, FateId fateId, @@ -218,7 +219,7 @@ private static Lock getLock(ServerContext context, AbstractId id, FateId fate byte[] lockData = fateId.canonical().getBytes(UTF_8); 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) { 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 51c4186bf9c..2881449fa88 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 @@ -96,7 +96,7 @@ public long isReady(FateId fateId, Manager manager) throws Exception { return 0; } - ZooReaderWriter zoo = manager.getContext().getZooReaderWriter(); + ZooReaderWriter zoo = manager.getContext().getZooSession().asReaderWriter(); if (isCancelled(fateId, manager.getContext())) { // compaction was canceled 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 ce8dbfcff85..6ce86cd47b3 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 @@ -63,7 +63,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 2451d31c345..8b78568fec6 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 @@ -54,7 +54,7 @@ public Repo call(FateId fateId, 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 7949af46bff..f7caa953f87 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 @@ -53,7 +53,7 @@ public RenameNamespace(NamespaceId namespaceId, String oldName, String newName) @Override public Repo call(FateId fateId, 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 1d5a9ff52f1..ceaae0804b0 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 @@ -75,7 +75,7 @@ public Repo call(FateId fateId, 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 3f66f14d8a1..e4cd9b39c7a 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 @@ -97,7 +97,7 @@ public long isReady(FateId fateId, Manager manager) { public Repo call(FateId fateId, 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 path = manager.getContext().getServerPaths().createTabletServerPath(resourceGroup, 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 611f035491f..13a8e8f67d6 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 @@ -319,8 +319,8 @@ private void abortIfFateTransactions(ServerContext context) { // as tablets are not assigned when this is called. The Fate code is not used to read from // zookeeper below because the serialization format changed in zookeeper, that is why a direct // read is performed. - if (!context.getZooReader().getChildren(context.getZooKeeperRoot() + Constants.ZFATE) - .isEmpty()) { + if (!context.getZooSession().asReader() + .getChildren(context.getZooKeeperRoot() + Constants.ZFATE).isEmpty()) { throw new AccumuloException("Aborting upgrade because there are" + " outstanding FATE transactions from a previous Accumulo version." + " You can start the tservers and then use the shell to delete completed " 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 ca3e72dc536..89a0713ff0b 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/main/java/org/apache/accumulo/manager/upgrade/Upgrader12to13.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader12to13.java index 91dea17cd22..d2d73dd6364 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader12to13.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader12to13.java @@ -118,7 +118,7 @@ public void upgradeMetadata(ServerContext context) { private static void addCompactionsNode(ServerContext context) { try { - context.getZooReaderWriter().putPersistentData( + context.getZooSession().asReaderWriter().putPersistentData( ZooUtil.getRoot(context.getInstanceID()) + Constants.ZCOMPACTIONS, new byte[0], ZooUtil.NodeExistsPolicy.SKIP); } catch (KeeperException | InterruptedException e) { @@ -152,7 +152,7 @@ private void removeCompactColumnsFromRootTabletMetadata(ServerContext context) { var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET; try { - var zrw = context.getZooReaderWriter(); + var zrw = context.getZooSession().asReaderWriter(); Stat stat = new Stat(); byte[] rootData = zrw.getData(rootBase, stat); @@ -245,7 +245,7 @@ private void removeBulkFileColumnsFromTable(ServerContext context, String tableN private void removeUnusedZKNodes(ServerContext context) { try { final String zkRoot = ZooUtil.getRoot(context.getInstanceID()); - final var zrw = context.getZooReaderWriter(); + final var zrw = context.getZooSession().asReaderWriter(); final String ZCOORDINATOR = "/coordinators"; final String BULK_ARBITRATOR_TYPE = "bulkTx"; @@ -356,7 +356,7 @@ private void validateEmptyZKWorkerServerPaths(ServerContext context) { // are empty. This means that for the Accumulo 4.0 upgrade, the Manager // should be started first before any other process. final String zkRoot = ZooUtil.getRoot(context.getInstanceID()); - final ZooReader zr = context.getZooReader(); + final ZooReader zr = context.getZooSession().asReader(); for (String serverPath : new String[] {Constants.ZCOMPACTORS, Constants.ZSSERVERS, Constants.ZTSERVERS, Constants.ZDEADTSERVERS}) { try { 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 11ca402c887..c95dcd9ad6a 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 @@ -37,115 +37,82 @@ import org.apache.accumulo.core.fate.FateId; import org.apache.accumulo.core.fate.FateInstanceType; 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.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; public class CompactionDriverTest { - public static class CancelledCompactionDriver extends CompactionDriver { + private static final InstanceId instance = InstanceId.of(UUID.randomUUID()); + private static final TableId tableId = TableId.of("testTable"); + private static class CompactionTestDriver extends CompactionDriver { private static final long serialVersionUID = 1L; + private final boolean cancelled; - public CancelledCompactionDriver(NamespaceId namespaceId, TableId tableId, byte[] startRow, - byte[] endRow) { - super(namespaceId, tableId, startRow, endRow); + static CompactionDriver cancelled() { + return new CompactionTestDriver(true); } - @Override - protected boolean isCancelled(FateId fateId, ServerContext context) - throws InterruptedException, KeeperException { - return true; + static CompactionDriver notCancelled() { + return new CompactionTestDriver(false); } - } - - public static class NotCancelledCompactionDriver extends CompactionDriver { - - private static final long serialVersionUID = 1L; - - public NotCancelledCompactionDriver(NamespaceId namespaceId, TableId tableId, byte[] startRow, - byte[] endRow) { - super(namespaceId, tableId, startRow, endRow); + private CompactionTestDriver(boolean cancelled) { + super(NamespaceId.of("testNamespace"), tableId, new byte[0], new byte[0]); + this.cancelled = cancelled; } @Override - protected boolean isCancelled(FateId fateId, ServerContext context) - throws InterruptedException, KeeperException { - return false; + protected boolean isCancelled(FateId fateId, ServerContext context) { + return cancelled; } - } - @Test - public void testCancelId() throws Exception { - - final InstanceId instance = InstanceId.of(UUID.randomUUID()); - final NamespaceId namespaceId = NamespaceId.of("13"); - final TableId tableId = TableId.of("42"); - final FateId compactionFateId = FateId.from(FateInstanceType.USER, UUID.randomUUID()); - final byte[] startRow = new byte[0]; - final byte[] endRow = new byte[0]; + private Manager manager; + private ServerContext ctx; + private ZooSession zk; - Manager manager = createMock(Manager.class); - ServerContext ctx = createMock(ServerContext.class); - ZooReaderWriter zrw = createMock(ZooReaderWriter.class); + @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(); + } - replay(manager, ctx, zrw); - - final CancelledCompactionDriver driver = - new CancelledCompactionDriver(namespaceId, tableId, startRow, endRow); - - var e = assertThrows(AcceptableThriftTableOperationException.class, - () -> driver.isReady(compactionFateId, 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(CompactionTestDriver.cancelled(), TableOperationsImpl.COMPACTION_CANCELED_MSG); } @Test public void testTableBeingDeleted() throws Exception { - - final InstanceId instance = InstanceId.of(UUID.randomUUID()); - final NamespaceId namespaceId = NamespaceId.of("14"); - final TableId tableId = TableId.of("43"); - final FateId compactionFateId = FateId.from(FateInstanceType.USER, UUID.randomUUID()); - 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(); - String deleteMarkerPath = PreDeleteTable.createDeleteMarkerPath(instance, tableId); - expect(zrw.exists(deleteMarkerPath)).andReturn(true); - - replay(manager, ctx, zrw); - - final NotCancelledCompactionDriver driver = - new NotCancelledCompactionDriver(namespaceId, tableId, startRow, endRow); + expect(zk.exists(deleteMarkerPath, null)).andReturn(new Stat()); + runDriver(CompactionTestDriver.notCancelled(), TableOperationsImpl.TABLE_DELETED_MSG); + } + private void runDriver(CompactionDriver driver, String expectedMessage) { + replay(manager, ctx, zk); var e = assertThrows(AcceptableThriftTableOperationException.class, - () -> driver.isReady(compactionFateId, manager)); - + () -> driver.isReady(FateId.from(FateInstanceType.USER, UUID.randomUUID()), 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 4d8b8a7889d..791548916a6 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 320c8459d23..cabd4ef957d 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 @@ -322,7 +322,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 = context.getServerPaths().createGarbageCollectorPath(); List locks = ServiceLock.validateAndSort(path, zk.getChildren(path.toString())); if (locks != null && !locks.isEmpty()) { @@ -398,7 +398,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)); @@ -713,7 +713,7 @@ private void getMonitorLock(HostAndPort monitorLocation) final var monitorLockPath = context.getServerPaths().createMonitorPath(); // 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) @@ -740,7 +740,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); HAServiceLockWatcher monitorLockWatcher = new HAServiceLockWatcher("monitor"); while (true) { 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 eb30a0375c0..9d414477b4a 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 @@ -215,7 +215,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 { final ServiceLockPath zLockPath = @@ -348,7 +348,7 @@ private ServiceLock announceExistence() { } throw e; } - 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 5e7f6286c58..08d83f148d7 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 @@ -231,7 +231,7 @@ protected TabletServer(ConfigOpts opts, Function serverContextFactory, String[] args) { super("tserver", opts, serverContextFactory, 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()); @@ -332,9 +332,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; } @@ -488,7 +487,7 @@ public ZooCache getManagerLockCache() { } private void announceExistence() { - ZooReaderWriter zoo = getContext().getZooReaderWriter(); + ZooReaderWriter zoo = getContext().getZooSession().asReaderWriter(); try { final ServiceLockPath zLockPath = @@ -509,7 +508,7 @@ private void announceExistence() { throw e; } 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 575a56a8ff1..549331b247a 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 @@ -621,7 +621,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); diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index 3d6a0e3dba3..82c0f9b8bce 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -237,7 +237,7 @@ public void testExternalCompaction() throws Exception { public void testCompactionCommitAndDeadDetectionRoot() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), createDummyLockID(), null); + ctx.getZooSession(), createDummyLockID(), null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var tableId = ctx.getTableId(AccumuloTable.ROOT.tableName()); @@ -256,7 +256,7 @@ public void testCompactionCommitAndDeadDetectionRoot() throws Exception { public void testCompactionCommitAndDeadDetectionMeta() throws Exception { var ctx = getCluster().getServerContext(); FateStore metaFateStore = new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, - ctx.getZooReaderWriter(), createDummyLockID(), null); + ctx.getZooSession(), createDummyLockID(), null); try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { // Metadata table by default already has 2 tablets @@ -302,9 +302,8 @@ public void testCompactionCommitAndDeadDetectionAll() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { UserFateStore userFateStore = new UserFateStore<>(ctx, createDummyLockID(), null); - FateStore metaFateStore = - new MetaFateStore<>(ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooReaderWriter(), - createDummyLockID(), null); + FateStore metaFateStore = new MetaFateStore<>( + ctx.getZooKeeperRoot() + Constants.ZFATE, ctx.getZooSession(), createDummyLockID(), null); SortedSet splits = new TreeSet<>(); splits.add(new Text(row(MAX_DATA / 2))); 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/FateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java index e72a5fb4217..0384bc97b1f 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/FateOpsCommandsIT.java @@ -645,7 +645,7 @@ protected void testFatePrintAndSummaryCommandsWithInProgressTxns(FateStore testMethod, int maxDeferred, FateIdGenerator fateIdGenerator) throws Exception { String zkRoot = FateStoreUtil.MetaFateZKSetup.getZkRoot(); - var zooReaderWriter = FateStoreUtil.MetaFateZKSetup.getZooReaderWriter(); + var zk = FateStoreUtil.MetaFateZKSetup.getZk(); String fatePath = FateStoreUtil.MetaFateZKSetup.getZkFatePath(); ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(zkRoot).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(zooReaderWriter).anyTimes(); + expect(sctx.getZooSession()).andReturn(zk).anyTimes(); replay(sctx); - testMethod.execute(new MetaFateStore<>(fatePath, zooReaderWriter, createDummyLockID(), null, - maxDeferred, fateIdGenerator), sctx); + testMethod.execute( + new MetaFateStore<>(fatePath, zk, createDummyLockID(), null, maxDeferred, fateIdGenerator), + sctx); } @Override protected TStatus getTxStatus(ServerContext sctx, FateId fateId) { try { - return getTxStatus(sctx.getZooReaderWriter(), fateId); + return getTxStatus(sctx.getZooSession(), fateId); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException(e); } @@ -84,9 +85,10 @@ protected TStatus getTxStatus(ServerContext sctx, FateId fateId) { * Get the status of the TX from ZK directly. Unable to call MetaFateStore.getStatus because this * test thread does not have the reservation (the FaTE thread does) */ - private static TStatus getTxStatus(ZooReaderWriter zrw, FateId fateId) + private static TStatus getTxStatus(ZooSession zk, FateId fateId) throws KeeperException, InterruptedException { String zkRoot = FateStoreUtil.MetaFateZKSetup.getZkRoot(); + var zrw = zk.asReaderWriter(); zrw.sync(zkRoot); String txdir = String.format("%s%s/tx_%s", zkRoot, Constants.ZFATE, fateId.getTxUUIDStr()); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java index af95d02b024..3e2cd3a2391 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateInterleavingIT.java @@ -26,7 +26,6 @@ import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; -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.test.fate.FateInterleavingIT; @@ -42,8 +41,8 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred AbstractFateStore.FateIdGenerator fateIdGenerator) throws Exception { ServerContext sctx = getCluster().getServerContext(); String path = ZK_ROOT + Constants.ZFATE; - ZooReaderWriter zk = sctx.getZooReaderWriter(); - zk.mkdirs(ZK_ROOT); + var zk = sctx.getZooSession(); + zk.asReaderWriter().mkdirs(ZK_ROOT); testMethod.execute(new MetaFateStore<>(path, zk, createDummyLockID(), null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java index c4c1e5b24a5..937a675f9c4 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateOpsCommandsIT.java @@ -23,7 +23,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.fate.AbstractFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.fate.FateOpsCommandsIT; @@ -33,7 +32,7 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, AbstractFateStore.FateIdGenerator fateIdGenerator) throws Exception { ServerContext sctx = getCluster().getServerContext(); String path = sctx.getZooKeeperRoot() + Constants.ZFATE; - ZooReaderWriter zk = sctx.getZooReaderWriter(); + var zk = sctx.getZooSession(); testMethod.execute(new MetaFateStore<>(path, zk, createDummyLockID(), null), sctx); } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStatusEnforcementIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStatusEnforcementIT.java index 4800e6816e6..d6e1410b7d5 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStatusEnforcementIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStatusEnforcementIT.java @@ -46,8 +46,7 @@ public static void afterAllTeardown() throws Exception { @BeforeEach public void beforeEachSetup() throws Exception { store = new MetaFateStore<>(FateStoreUtil.MetaFateZKSetup.getZkFatePath(), - FateStoreUtil.MetaFateZKSetup.getZooReaderWriter(), AbstractFateStore.createDummyLockID(), - null); + FateStoreUtil.MetaFateZKSetup.getZk(), AbstractFateStore.createDummyLockID(), null); fateId = store.create(); txStore = store.reserve(fateId); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java index 66d01b1489a..f8e89b101c3 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaFateStoreFateIT.java @@ -71,13 +71,11 @@ public void executeTest(FateTestExecutor testMethod, int maxDeferred, FateIdGenerator fateIdGenerator) throws Exception { ServerContext sctx = createMock(ServerContext.class); expect(sctx.getZooKeeperRoot()).andReturn(FateStoreUtil.MetaFateZKSetup.getZkRoot()).anyTimes(); - expect(sctx.getZooReaderWriter()).andReturn(FateStoreUtil.MetaFateZKSetup.getZooReaderWriter()) - .anyTimes(); + expect(sctx.getZooSession()).andReturn(FateStoreUtil.MetaFateZKSetup.getZk()).anyTimes(); replay(sctx); - MetaFateStore store = - new MetaFateStore<>(FateStoreUtil.MetaFateZKSetup.getZkFatePath(), - FateStoreUtil.MetaFateZKSetup.getZooReaderWriter(), createDummyLockID(), null, - maxDeferred, fateIdGenerator); + MetaFateStore store = new MetaFateStore<>( + FateStoreUtil.MetaFateZKSetup.getZkFatePath(), FateStoreUtil.MetaFateZKSetup.getZk(), + createDummyLockID(), null, maxDeferred, fateIdGenerator); // Check that the store has no transactions before and after each test assertEquals(0, store.list().count()); @@ -117,7 +115,7 @@ protected void deleteKey(FateId fateId, ServerContext sctx) { String txPath = FateStoreUtil.MetaFateZKSetup.getZkFatePath() + "/tx_" + fateId.getTxUUIDStr(); Object currentNode = serializedCons.newInstance( - new Object[] {FateStoreUtil.MetaFateZKSetup.getZooReaderWriter().getData(txPath)}); + new Object[] {FateStoreUtil.MetaFateZKSetup.getZk().asReader().getData(txPath)}); TStatus currentStatus = (TStatus) status.get(currentNode); Optional currentReservation = getCurrentReservation(reservation, currentNode); @@ -129,7 +127,7 @@ protected void deleteKey(FateId fateId, ServerContext sctx) { Object newNode = fateDataCons.newInstance(currentStatus, currentReservation.orElse(null), null, currentRepoDeque, currentTxInfo); - FateStoreUtil.MetaFateZKSetup.getZooReaderWriter().putPersistentData(txPath, + FateStoreUtil.MetaFateZKSetup.getZk().asReaderWriter().putPersistentData(txPath, (byte[]) nodeSerialize.invoke(newNode), NodeExistsPolicy.OVERWRITE); } catch (Exception e) { throw new IllegalStateException(e); diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaMultipleStoresIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaMultipleStoresIT.java index d5d1903493b..b7d11278737 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaMultipleStoresIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/MetaMultipleStoresIT.java @@ -62,7 +62,7 @@ static class SleepingEnvMetaStoreFactory implements TestStoreFactory create(ZooUtil.LockID lockID, Predicate isLockHeld) throws InterruptedException, KeeperException { return new MetaFateStore<>(FateStoreUtil.MetaFateZKSetup.getZkFatePath(), - FateStoreUtil.MetaFateZKSetup.getZooReaderWriter(), lockID, isLockHeld); + FateStoreUtil.MetaFateZKSetup.getZk(), lockID, isLockHeld); } } @@ -71,7 +71,7 @@ static class LatchEnvMetaStoreFactory implements TestStoreFactory public FateStore create(ZooUtil.LockID lockID, Predicate isLockHeld) throws InterruptedException, KeeperException { return new MetaFateStore<>(FateStoreUtil.MetaFateZKSetup.getZkFatePath(), - FateStoreUtil.MetaFateZKSetup.getZooReaderWriter(), lockID, isLockHeld); + FateStoreUtil.MetaFateZKSetup.getZk(), lockID, isLockHeld); } } } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/meta/ZooMutatorIT.java b/test/src/main/java/org/apache/accumulo/test/fate/meta/ZooMutatorIT.java index fdfb0ec37e8..f3e67b8dc6d 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/meta/ZooMutatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/meta/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 fd8ccb90883..5f788cd18fe 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 @@ -44,7 +44,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(); // wait for 2 lock entries var path = getServerContext().getServerPaths().createManagerPath(); @@ -58,7 +58,7 @@ public void test() throws Exception { ServiceLock.validateAndSort(path, writer.getChildren(path.toString())); String lockPath = path + "/" + 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 f74944692e8..33d1bd33187 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 @@ -46,7 +46,6 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.Namespace; -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; @@ -55,12 +54,12 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; -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.util.compaction.ExternalCompactionUtil; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.test.util.SlowOps; import org.apache.accumulo.test.util.Wait; import org.apache.zookeeper.KeeperException; @@ -90,12 +89,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; @@ -108,8 +105,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); } @@ -262,7 +258,7 @@ public void getFateStatus() { try { InstanceId instanceId = context.getInstanceID(); - ZooReaderWriter zk = context.getZooReader().asWriter(secret); + var zk = context.getZooSession(); MetaFateStore mfs = new MetaFateStore<>( ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createDummyLockID(), null); UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), null); @@ -355,7 +351,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(); MetaFateStore mfs = new MetaFateStore<>(ZooUtil.getRoot(instanceId) + Constants.ZFATE, zk, createDummyLockID(), null); var lockPath = context.getServerPaths().createTableLocksPath(tableId.toString()); 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 28b08dbbf02..ee4b187a61f 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 @@ -65,7 +65,6 @@ import org.apache.accumulo.core.fate.ReadOnlyFateStore; import org.apache.accumulo.core.fate.user.UserFateStore; import org.apache.accumulo.core.fate.zookeeper.MetaFateStore; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; import org.apache.accumulo.core.metadata.AccumuloTable; import org.apache.accumulo.core.metadata.StoredTabletFile; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; @@ -233,7 +232,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(); MetaFateStore mfs = new MetaFateStore<>(context.getZooKeeperRoot() + Constants.ZFATE, zk, createDummyLockID(), null); UserFateStore ufs = new UserFateStore<>(context, createDummyLockID(), 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 ff0c4a35a4a..1d1fa0425cb 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 @@ -116,7 +116,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 = getServerContext().getServerPaths().createGarbageCollectorPath(); - ZooReaderWriter zk = getServerContext().getZooReaderWriter(); + ZooReaderWriter zk = getServerContext().getZooSession().asReaderWriter(); try { ServiceLock.deleteLock(zk, path); } catch (IllegalStateException e) { @@ -410,7 +410,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 = getServerContext().getServerPaths().createGarbageCollectorPath(); for (int i = 0; i < 5; i++) { List locks; 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 3f791f411bd..123d611b495 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 @@ -104,7 +104,7 @@ public void evaluateOnDemandTabletsForUnload() { 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().asReader().exists(tableZPath, new StuckWatcher()); } catch (KeeperException | InterruptedException e) { LOG.error("Error setting watch at: {}", tableZPath, e); } @@ -192,7 +192,7 @@ public boolean testTabletServerWithStuckWatcherDies() throws Exception { (rg) -> rg.equals(Constants.DEFAULT_RESOURCE_GROUP_NAME), AddressSelector.exact(HostAndPort.fromString(tserver.toHostPortString())), true); assertEquals(1, serverPaths.size()); - ctx.getZooReaderWriter().recursiveDelete(serverPaths.iterator().next().toString(), + ctx.getZooSession().asReaderWriter().recursiveDelete(serverPaths.iterator().next().toString(), NodeMissingPolicy.FAIL); Wait.waitFor(() -> pingServer(client, tserver.toHostPortString()) == 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/WatchTheWatchCountIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java index b90d8b7c6ea..e5550308055 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 5477968edc1..2155fc93e03 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 @@ -133,10 +133,10 @@ public static void main(String[] args) throws Exception { var zLockPath = context.getServerPaths() .createTabletServerPath(Constants.DEFAULT_RESOURCE_GROUP_NAME, serverPort.address); - 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 3380402168e..af97575f23d 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 @@ -39,19 +39,17 @@ import java.util.concurrent.locks.LockSupport; import org.apache.accumulo.core.Constants; -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; import org.apache.accumulo.core.lock.ServiceLockData; import org.apache.accumulo.core.lock.ServiceLockData.ThriftService; import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath; +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; @@ -76,23 +74,29 @@ protected TestServiceLockPath(String path) { @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) @@ -183,7 +187,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) { @@ -201,14 +205,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(); @@ -258,8 +262,8 @@ public void testDeleteLock() throws Exception { var parent = new TestServiceLockPath( "/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()); @@ -279,7 +283,7 @@ public void testDeleteLock() throws Exception { assertNull(lw.exception); assertNull(lw.reason); - zk.delete(zl.getLockPath()); + zrw.delete(zl.getLockPath()); lw.waitForChanges(2); @@ -294,8 +298,8 @@ public void testDeleteWaiting() throws Exception { var parent = new TestServiceLockPath( "/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()); @@ -333,9 +337,9 @@ public void testDeleteWaiting() throws Exception { zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), "test3", ThriftService.TSERV, Constants.DEFAULT_RESOURCE_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); @@ -343,7 +347,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); @@ -369,7 +373,7 @@ public void testUnexpectedEvent() throws Exception { var parent = new TestServiceLockPath( "/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); @@ -413,8 +417,8 @@ public void testUnexpectedEvent() throws Exception { public void testLockSerial() throws Exception { var parent = new TestServiceLockPath("/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, @@ -458,7 +462,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"); @@ -520,17 +524,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, - Constants.DEFAULT_RESOURCE_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, + Constants.DEFAULT_RESOURCE_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; @@ -567,7 +569,7 @@ private int parseLockWorkerName(String child) { public void testLockParallel() throws Exception { var parent = new TestServiceLockPath("/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); @@ -598,7 +600,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); @@ -617,7 +619,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); } @@ -632,7 +634,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, @@ -666,7 +668,7 @@ public void testChangeData() throws Exception { var parent = new TestServiceLockPath( "/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 9ddff41e0dd..6335a8dd620 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 @@ -74,6 +74,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService; import org.apache.accumulo.core.tabletserver.thrift.TabletStats; import org.apache.accumulo.core.util.threads.ThreadPools; +import org.apache.accumulo.core.zookeeper.ZooSession; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.client.ClientServiceHandler; import org.apache.accumulo.server.manager.state.Assignment; @@ -84,7 +85,6 @@ import org.apache.thrift.TException; import org.apache.thrift.TMultiplexedProcessor; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -291,7 +291,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); NullTServerTabletClientHandler tch = new NullTServerTabletClientHandler(); @@ -343,11 +343,11 @@ public void failedToAcquireLock(Exception e) { ServiceLock miniLock = null; try { - ZooKeeper zk = context.getZooReaderWriter().getZooKeeper(); + ZooSession zk = context.getZooSession(); UUID nullTServerUUID = UUID.randomUUID(); ServiceLockPath slp = context.getServerPaths().createMiniPath(nullTServerUUID.toString()); try { - context.getZooReaderWriter().mkdirs(slp.toString()); + zk.asReaderWriter().mkdirs(slp.toString()); } catch (KeeperException | InterruptedException e) { throw new IllegalStateException("Error creating path in ZooKeeper", e); } diff --git a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooCacheIT.java b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooCacheIT.java index 645b7c70d2f..ef15a309598 100644 --- a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooCacheIT.java +++ b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooCacheIT.java @@ -29,10 +29,13 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; 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.zookeeper.ZooSession; import org.apache.accumulo.test.util.Wait; import org.apache.zookeeper.Watcher; import org.junit.jupiter.api.AfterEach; @@ -44,8 +47,9 @@ @Tag(ZOOKEEPER_TESTING_SERVER) public class ZooCacheIT { - private ZooKeeperTestingServer szk = null; - private ZooReaderWriter zk = null; + private ZooKeeperTestingServer szk; + private ZooSession zk; + private ZooReaderWriter zrw; @TempDir private File tempDir; @@ -53,29 +57,34 @@ public class ZooCacheIT { @BeforeEach public void setup() throws Exception { szk = new ZooKeeperTestingServer(tempDir); - zk = szk.getZooReaderWriter(); + zk = szk.newClient(); + zrw = zk.asReaderWriter(); } @AfterEach public void teardown() throws Exception { - szk.close(); + try { + zk.close(); + } finally { + szk.close(); + } } @Test public void testGetChildren() throws Exception { Set watchesRemoved = Collections.synchronizedSet(new HashSet<>()); - Watcher watcher = event -> { + ZooCacheWatcher watcher = event -> { if (event.getType() == Watcher.Event.EventType.ChildWatchRemoved || event.getType() == Watcher.Event.EventType.DataWatchRemoved) { watchesRemoved.add(event.getPath()); } }; - ZooCache zooCache = new ZooCache(zk, watcher, Duration.ofSeconds(3)); + ZooCache zooCache = new ZooCache(zk, Optional.of(watcher), Duration.ofSeconds(3)); - zk.mkdirs("/test2"); - zk.mkdirs("/test3/c1"); - zk.mkdirs("/test3/c2"); + zrw.mkdirs("/test2"); + zrw.mkdirs("/test3/c1"); + zrw.mkdirs("/test3/c2"); // cache non-existence of /test1 and existence of /test2 and /test3 long uc1 = zooCache.getUpdateCount(); @@ -98,7 +107,7 @@ public void testGetChildren() throws Exception { assertEquals(uc4, zooCache.getUpdateCount()); // Had cached non-existence of "/test1", should get a notification that it was created - zk.mkdirs("/test1"); + zrw.mkdirs("/test1"); Wait.waitFor(() -> { var children = zooCache.getChildren("/test1"); @@ -113,7 +122,7 @@ public void testGetChildren() throws Exception { assertEquals(uc5, zooCache.getUpdateCount()); // add a child to /test3, should get a notification of the change - zk.mkdirs("/test3/c3"); + zrw.mkdirs("/test3/c3"); Wait.waitFor(() -> { var children = zooCache.getChildren("/test3"); return children != null && children.size() == 3; @@ -126,7 +135,7 @@ public void testGetChildren() throws Exception { assertEquals(uc6, zooCache.getUpdateCount()); // remove a child from /test3 - zk.delete("/test3/c2"); + zrw.delete("/test3/c2"); Wait.waitFor(() -> { var children = zooCache.getChildren("/test3"); return children != null && children.size() == 2; @@ -139,7 +148,7 @@ public void testGetChildren() throws Exception { assertEquals(uc7, zooCache.getUpdateCount()); // remove /test2, should start caching that it does not exist - zk.delete("/test2"); + zrw.delete("/test2"); Wait.waitFor(() -> zooCache.getChildren("/test2") == null); long uc8 = zooCache.getUpdateCount(); assertTrue(uc7 < uc8); @@ -149,7 +158,7 @@ public void testGetChildren() throws Exception { assertEquals(uc8, zooCache.getUpdateCount()); // add /test2 back, should update - zk.mkdirs("/test2"); + zrw.mkdirs("/test2"); Wait.waitFor(() -> zooCache.getChildren("/test2") != null); long uc9 = zooCache.getUpdateCount(); assertTrue(uc8 < uc9); @@ -159,12 +168,12 @@ public void testGetChildren() throws Exception { assertEquals(uc9, zooCache.getUpdateCount()); // make multiple changes. the cache should see all of these - zk.delete("/test1"); - zk.mkdirs("/test2/ca"); - zk.delete("/test3/c1"); - zk.mkdirs("/test3/c4"); - zk.delete("/test3/c4"); - zk.mkdirs("/test3/c5"); + zrw.delete("/test1"); + zrw.mkdirs("/test2/ca"); + zrw.delete("/test3/c1"); + zrw.mkdirs("/test3/c4"); + zrw.delete("/test3/c4"); + zrw.mkdirs("/test3/c5"); Wait.waitFor(() -> { var children1 = zooCache.getChildren("/test1"); 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