Skip to content

Commit

Permalink
Remove unused vars and use ServerContext in ZooZap
Browse files Browse the repository at this point in the history
  • Loading branch information
ctubbsii committed Dec 16, 2024
1 parent 9626c59 commit 434748f
Show file tree
Hide file tree
Showing 7 changed files with 56 additions and 74 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
116 changes: 56 additions & 60 deletions server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,15 +38,13 @@
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;

import org.apache.accumulo.core.Constants;
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;
Expand Down Expand Up @@ -180,7 +178,6 @@ public Repo<Manager> 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");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -249,7 +248,6 @@ public void getFateStatus() {

try {

InstanceId instanceId = context.getInstanceID();
ZooReaderWriter zk = context.getZooReader().asWriter(secret);
ZooStore<String> zs = new ZooStore<>(Constants.ZFATE, zk);
var lockPath = ServiceLock.path(Constants.ZTABLE_LOCKS + "/" + tableId);
Expand Down Expand Up @@ -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<String> zs = new ZooStore<>(Constants.ZFATE, zk);
var lockPath = ServiceLock.path(Constants.ZTABLE_LOCKS + "/" + tableId);
Expand Down

0 comments on commit 434748f

Please sign in to comment.