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 6c9d905a338..5d8146a61e6 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 @@ -27,7 +27,6 @@ import java.util.Map.Entry; import java.util.UUID; -import org.apache.accumulo.core.clientImpl.ClientContext; 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; @@ -86,13 +85,11 @@ public enum WalState { UNREFERENCED } - private final ClientContext context; private final ZooReaderWriter zoo; private volatile boolean checkedExistance = false; public WalStateManager(ServerContext context) { - this.context = context; this.zoo = context.getZooReaderWriter(); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java index fb04befb616..ca298423e4b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java @@ -54,13 +54,11 @@ public abstract class TabletMutatorBase implements Ample.TabletMutator { - private final ServerContext context; private final Mutation mutation; protected AutoCloseable closeAfterMutate; private boolean updatesEnabled = true; protected TabletMutatorBase(ServerContext context, KeyExtent extent) { - this.context = context; mutation = new Mutation(extent.toMetaRow()); } 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 7270b826ef7..97db98e4ea0 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 @@ -19,7 +19,6 @@ package org.apache.accumulo.server.util; import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Objects.requireNonNull; import static org.apache.accumulo.core.fate.FateTxId.parseTidFromUserInput; import java.io.BufferedWriter; 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 aed415e72c0..c0e76b71e4c 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 @@ -24,18 +24,14 @@ import org.apache.accumulo.core.cli.Help; 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.ZooReaderWriter; import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy; import org.apache.accumulo.core.lock.ServiceLock; import org.apache.accumulo.core.singletons.SingletonManager; import org.apache.accumulo.core.singletons.SingletonManager.Mode; -import org.apache.accumulo.core.volume.VolumeConfiguration; -import org.apache.accumulo.server.fs.VolumeManager; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.security.SecurityUtil; import org.apache.accumulo.start.spi.KeywordExecutable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,81 +97,81 @@ public void execute(String[] args) throws Exception { SecurityUtil.serverLogin(siteConf); } - String volDir = VolumeConfiguration.getVolumeUris(siteConf).iterator().next(); - Path instanceDir = new Path(volDir, "instance_id"); - InstanceId iid = VolumeManager.getInstanceIDFromHdfs(instanceDir, new Configuration()); - ZooReaderWriter zoo = new ZooReaderWriter(siteConf); + try (var context = new ServerContext(siteConf)) { + ZooReaderWriter zoo = context.getZooReaderWriter(); - if (opts.zapManager) { - try { - zapDirectory(zoo, Constants.ZMANAGER_LOCK, opts); - } catch (KeeperException | InterruptedException e) { - e.printStackTrace(); + if (opts.zapManager) { + try { + zapDirectory(zoo, Constants.ZMANAGER_LOCK, opts); + } catch (KeeperException | InterruptedException e) { + e.printStackTrace(); + } } - } - if (opts.zapTservers) { - try { - List children = zoo.getChildren(Constants.ZTSERVERS); - for (String child : children) { - message("Deleting " + Constants.ZTSERVERS + "/" + child + " from zookeeper", opts); - - if (opts.zapManager) { - zoo.recursiveDelete(Constants.ZTSERVERS + "/" + child, NodeMissingPolicy.SKIP); - } else { - var zLockPath = ServiceLock.path(Constants.ZTSERVERS + "/" + child); - if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { - if (!ServiceLock.deleteLock(zoo, zLockPath, "tserver")) { - message("Did not delete " + Constants.ZTSERVERS + "/" + child, opts); + if (opts.zapTservers) { + try { + List children = zoo.getChildren(Constants.ZTSERVERS); + for (String child : children) { + message("Deleting " + Constants.ZTSERVERS + "/" + child + " from zookeeper", opts); + + if (opts.zapManager) { + zoo.recursiveDelete(Constants.ZTSERVERS + "/" + child, NodeMissingPolicy.SKIP); + } else { + var zLockPath = ServiceLock.path(Constants.ZTSERVERS + "/" + child); + if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { + if (!ServiceLock.deleteLock(zoo, zLockPath, "tserver")) { + message("Did not delete " + Constants.ZTSERVERS + "/" + child, opts); + } } } } + } catch (KeeperException | InterruptedException e) { + log.error("{}", e.getMessage(), e); } - } catch (KeeperException | InterruptedException e) { - log.error("{}", e.getMessage(), e); } - } - if (opts.zapCoordinators) { - try { - if (zoo.exists(Constants.ZCOORDINATOR_LOCK)) { - zapDirectory(zoo, Constants.ZCOORDINATOR_LOCK, opts); + if (opts.zapCoordinators) { + try { + if (zoo.exists(Constants.ZCOORDINATOR_LOCK)) { + zapDirectory(zoo, Constants.ZCOORDINATOR_LOCK, opts); + } + } catch (KeeperException | InterruptedException e) { + log.error("Error deleting coordinator from zookeeper, {}", e.getMessage(), e); } - } catch (KeeperException | InterruptedException e) { - log.error("Error deleting coordinator from zookeeper, {}", e.getMessage(), e); } - } - if (opts.zapCompactors) { - try { - if (zoo.exists(Constants.ZCOMPACTORS)) { - List queues = zoo.getChildren(Constants.ZCOMPACTORS); - for (String queue : queues) { - message("Deleting " + Constants.ZCOMPACTORS + "/" + queue + " from zookeeper", opts); - zoo.recursiveDelete(Constants.ZCOMPACTORS + "/" + queue, NodeMissingPolicy.SKIP); + if (opts.zapCompactors) { + try { + if (zoo.exists(Constants.ZCOMPACTORS)) { + List queues = zoo.getChildren(Constants.ZCOMPACTORS); + for (String queue : queues) { + message("Deleting " + Constants.ZCOMPACTORS + "/" + queue + " from zookeeper", + opts); + zoo.recursiveDelete(Constants.ZCOMPACTORS + "/" + queue, NodeMissingPolicy.SKIP); + } } + } catch (KeeperException | InterruptedException e) { + log.error("Error deleting compactors from zookeeper, {}", e.getMessage(), e); } - } catch (KeeperException | InterruptedException e) { - log.error("Error deleting compactors from zookeeper, {}", e.getMessage(), e); - } - } + } - if (opts.zapScanServers) { - try { - if (zoo.exists(Constants.ZSSERVERS)) { - List children = zoo.getChildren(Constants.ZSSERVERS); - for (String child : children) { - message("Deleting " + Constants.ZSSERVERS + "/" + child + " from zookeeper", opts); + if (opts.zapScanServers) { + try { + if (zoo.exists(Constants.ZSSERVERS)) { + List children = zoo.getChildren(Constants.ZSSERVERS); + for (String child : children) { + message("Deleting " + Constants.ZSSERVERS + "/" + child + " from zookeeper", opts); - var zLockPath = ServiceLock.path(Constants.ZSSERVERS + "/" + child); - if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { - ServiceLock.deleteLock(zoo, zLockPath); + var zLockPath = ServiceLock.path(Constants.ZSSERVERS + "/" + child); + if (!zoo.getChildren(zLockPath.toString()).isEmpty()) { + ServiceLock.deleteLock(zoo, zLockPath); + } } } + } catch (KeeperException | InterruptedException e) { + log.error("{}", e.getMessage(), e); } - } catch (KeeperException | InterruptedException e) { - log.error("{}", e.getMessage(), e); } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java index bdc4486882a..ee7036c8fff 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java @@ -57,11 +57,9 @@ public interface Arbitrator { public static class ZooArbitrator implements Arbitrator { - private final ServerContext context; private final ZooReader rdr; public ZooArbitrator(ServerContext context) { - this.context = context; rdr = context.getZooReader(); } diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java index e26fd09e3b4..69b49af3830 100644 --- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java @@ -38,7 +38,6 @@ import java.io.File; import java.util.ArrayList; import java.util.List; -import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -46,7 +45,6 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperation; import org.apache.accumulo.core.conf.ConfigurationCopy; 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.data.TableId; import org.apache.accumulo.core.fate.AgeOffStore; @@ -180,7 +178,6 @@ public Repo call(long tid, Manager environment) throws Exception { private static ZooKeeperTestingServer szk = null; private static ZooReaderWriter zk = null; - private static final InstanceId IID = InstanceId.of(UUID.randomUUID()); private static final NamespaceId NS = NamespaceId.of("testNameSpace"); private static final TableId TID = TableId.of("testTable"); 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 5441cd1c75d..dcfcba3b9f3 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 @@ -43,7 +43,6 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.data.InstanceId; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.fate.AdminUtil; import org.apache.accumulo.core.fate.ZooStore; @@ -249,7 +248,6 @@ public void getFateStatus() { try { - InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); ZooStore zs = new ZooStore<>(Constants.ZFATE, zk); var lockPath = ServiceLock.path(Constants.ZTABLE_LOCKS + "/" + tableId); @@ -338,7 +336,6 @@ private boolean lookupFateInZookeeper(final String tableName) throws KeeperExcep log.trace("tid: {}", tableId); - InstanceId instanceId = context.getInstanceID(); ZooReaderWriter zk = context.getZooReader().asWriter(secret); ZooStore zs = new ZooStore<>(Constants.ZFATE, zk); var lockPath = ServiceLock.path(Constants.ZTABLE_LOCKS + "/" + tableId);