Skip to content

Commit

Permalink
Merge branch 'main' into common-zk-path-creation
Browse files Browse the repository at this point in the history
  • Loading branch information
dlmarion committed Jan 6, 2025
2 parents a8c933d + 671675c commit da51e06
Show file tree
Hide file tree
Showing 20 changed files with 249 additions and 240 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,19 @@
*/
package org.apache.accumulo.core.lock;

import java.util.function.Consumer;
import java.util.function.Supplier;

import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
import org.apache.accumulo.core.lock.ServiceLock.AccumuloLockWatcher;
import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath;
import org.apache.accumulo.core.util.Halt;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoAuthException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -51,4 +59,122 @@ public static void createNonHaServiceLockPath(Type server, ZooReaderWriter zrw,
}

}

/**
* Lock Watcher used by Highly Available services. These are services where only instance is
* running at a time, but another backup service can be started that will be used if the active
* service instance fails and loses its lock in ZK.
*/
public static class HAServiceLockWatcher implements AccumuloLockWatcher {

private static final Logger LOG = LoggerFactory.getLogger(HAServiceLockWatcher.class);

private final Type server;
private volatile boolean acquiredLock = false;
private volatile boolean failedToAcquireLock = false;

public HAServiceLockWatcher(Type server) {
this.server = server;
}

@Override
public void lostLock(LockLossReason reason) {
Halt.halt(server + " lock in zookeeper lost (reason = " + reason + "), exiting!", -1);
}

@Override
public void unableToMonitorLockNode(final Exception e) {
// ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
Halt.halt(-1, () -> LOG.error("FATAL: No longer able to monitor {} lock node", server, e));

}

@Override
public synchronized void acquiredLock() {
LOG.debug("Acquired {} lock", server);

if (acquiredLock || failedToAcquireLock) {
Halt.halt("Zoolock in unexpected state AL " + acquiredLock + " " + failedToAcquireLock, -1);
}

acquiredLock = true;
notifyAll();
}

@Override
public synchronized void failedToAcquireLock(Exception e) {
LOG.warn("Failed to get {} lock", server, e);

if (e instanceof NoAuthException) {
String msg =
"Failed to acquire " + server + " lock due to incorrect ZooKeeper authentication.";
LOG.error("{} Ensure instance.secret is consistent across Accumulo configuration", msg, e);
Halt.halt(msg, -1);
}

if (acquiredLock) {
Halt.halt("Zoolock in unexpected state acquiredLock true with FAL " + failedToAcquireLock,
-1);
}

failedToAcquireLock = true;
notifyAll();
}

public synchronized void waitForChange() {
while (!acquiredLock && !failedToAcquireLock) {
try {
LOG.info("{} lock held by someone else, waiting for a change in state", server);
wait();
} catch (InterruptedException e) {
// empty
}
}
}

public boolean isLockAcquired() {
return acquiredLock;
}

public boolean isFailedToAcquireLock() {
return failedToAcquireLock;
}

}

/**
* Lock Watcher used by non-HA services
*/
public static class ServiceLockWatcher implements LockWatcher {

private static final Logger LOG = LoggerFactory.getLogger(ServiceLockWatcher.class);

private final Type server;
private final Supplier<Boolean> shuttingDown;
private final Consumer<Type> lostLockAction;

public ServiceLockWatcher(Type server, Supplier<Boolean> shuttingDown,
Consumer<Type> lostLockAction) {
this.server = server;
this.shuttingDown = shuttingDown;
this.lostLockAction = lostLockAction;
}

@Override
public void lostLock(final LockLossReason reason) {
Halt.halt(1, () -> {
if (!shuttingDown.get()) {
LOG.error("{} lost lock (reason = {}), exiting.", server, reason);
}
lostLockAction.accept(server);
});
}

@Override
public void unableToMonitorLockNode(final Exception e) {
Halt.halt(1, () -> LOG.error("Lost ability to monitor {} lock, exiting.", server, e));
}

}

}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.Predicate;
import java.util.function.Supplier;

import org.apache.accumulo.core.client.ConditionalWriter;
import org.apache.accumulo.core.client.admin.TabletAvailability;
Expand Down Expand Up @@ -623,6 +624,16 @@ ConditionalTabletMutator requireSame(TabletMetadata tabletMetadata, ColumnType t
* let the rejected status carry forward in this case.
*/
void submit(RejectionHandler rejectionHandler);

/**
* Overloaded version of {@link #submit(RejectionHandler)} that takes a short description of the
* operation to assist with debugging.
*
* @param rejectionHandler The rejection handler
* @param description A short description of the operation (e.g., "bulk import", "compaction")
*/
void submit(RejectionHandler rejectionHandler, Supplier<String> description);

}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Supplier;

import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.admin.TabletAvailability;
Expand Down Expand Up @@ -86,17 +87,20 @@ public class ConditionalTabletMutatorImpl extends TabletMutatorBase<Ample.Condit
private final ServerContext context;
private final ServiceLock lock;
private final KeyExtent extent;
private final BiConsumer<KeyExtent,Supplier<String>> descriptionConsumer;

private boolean sawOperationRequirement = false;
private boolean checkPrevEndRow = true;

protected ConditionalTabletMutatorImpl(ServerContext context, KeyExtent extent,
Consumer<ConditionalMutation> mutationConsumer,
BiConsumer<KeyExtent,Ample.RejectionHandler> rejectionHandlerConsumer) {
BiConsumer<KeyExtent,Ample.RejectionHandler> rejectionHandlerConsumer,
BiConsumer<KeyExtent,Supplier<String>> descriptionConsumer) {
super(new ConditionalMutation(extent.toMetaRow()));
this.mutation = (ConditionalMutation) super.mutation;
this.mutationConsumer = mutationConsumer;
this.rejectionHandlerConsumer = rejectionHandlerConsumer;
this.descriptionConsumer = descriptionConsumer;
this.extent = extent;
this.context = context;
this.lock = this.context.getServiceLock();
Expand Down Expand Up @@ -390,4 +394,10 @@ public void submit(Ample.RejectionHandler rejectionCheck) {
mutationConsumer.accept(mutation);
rejectionHandlerConsumer.accept(extent, rejectionCheck);
}

@Override
public void submit(Ample.RejectionHandler rejectionHandler, Supplier<String> description) {
descriptionConsumer.accept(extent, description);
this.submit(rejectionHandler);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;

import org.apache.accumulo.core.client.AccumuloException;
Expand Down Expand Up @@ -66,6 +67,7 @@ public class ConditionalTabletsMutatorImpl implements Ample.ConditionalTabletsMu
private boolean active = true;

final Map<KeyExtent,Ample.RejectionHandler> rejectedHandlers = new HashMap<>();
private final Map<KeyExtent,Supplier<String>> operationDescriptions = new HashMap<>();
private final Function<DataLevel,String> tableMapper;

public ConditionalTabletsMutatorImpl(ServerContext context) {
Expand Down Expand Up @@ -93,7 +95,8 @@ public Ample.OperationRequirements mutateTablet(KeyExtent extent) {

Preconditions.checkState(extents.putIfAbsent(extent.toMetaRow(), extent) == null,
"Duplicate extents not handled %s", extent);
return new ConditionalTabletMutatorImpl(context, extent, mutations::add, rejectedHandlers::put);
return new ConditionalTabletMutatorImpl(context, extent, mutations::add, rejectedHandlers::put,
operationDescriptions::put);
}

protected ConditionalWriter createConditionalWriter(Ample.DataLevel dataLevel)
Expand Down Expand Up @@ -262,16 +265,20 @@ public Status getStatus() {
status = Status.ACCEPTED;
}

Supplier<String> descSupplier = operationDescriptions.get(extent);
String desc = (descSupplier == null) ? null : descSupplier.get();

if (log.isTraceEnabled()) {
// log detailed info about tablet metadata and mutation
log.trace("Mutation was rejected, status:{} {} {}", status, tabletMetadata,
result.getMutation().prettyPrint());
log.trace("Mutation was rejected, status:{}. Operation description: {} {} {}",
status, desc, tabletMetadata, result.getMutation().prettyPrint());
} else if (log.isDebugEnabled()) {
// log a single line of info that makes it apparent this happened and gives enough
// information to investigate
log.debug("Mutation was rejected, status:{} extent:{} row:{}", status,
tabletMetadata == null ? null : tabletMetadata.getExtent(),
new String(result.getMutation().getRow(), UTF_8));
log.debug(
"Mutation was rejected, status:{} extent:{} row:{} operation description: {}",
status, tabletMetadata == null ? null : tabletMetadata.getExtent(),
new String(result.getMutation().getRow(), UTF_8), desc);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,14 +75,14 @@
import org.apache.accumulo.core.file.FileSKVIterator;
import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
import org.apache.accumulo.core.lock.ServiceLock;
import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
import org.apache.accumulo.core.lock.ServiceLockData;
import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors;
import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
import org.apache.accumulo.core.lock.ServiceLockPaths.ServiceLockPath;
import org.apache.accumulo.core.lock.ServiceLockSupport;
import org.apache.accumulo.core.lock.ServiceLockSupport.ServiceLockWatcher;
import org.apache.accumulo.core.manager.state.tables.TableState;
import org.apache.accumulo.core.metadata.ReferencedTabletFile;
import org.apache.accumulo.core.metadata.StoredTabletFile;
Expand All @@ -101,7 +101,6 @@
import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.util.Halt;
import org.apache.accumulo.core.util.Timer;
import org.apache.accumulo.core.util.UtilWaitThread;
import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
Expand Down Expand Up @@ -281,20 +280,8 @@ protected void announceExistence(HostAndPort clientAddress)
ServiceLockSupport.createNonHaServiceLockPath(Type.COMPACTOR, zoo, path);
compactorLock =
new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, compactorId);
LockWatcher lw = new LockWatcher() {
@Override
public void lostLock(final LockLossReason reason) {
Halt.halt(1, () -> {
LOG.error("Compactor lost lock (reason = {}), exiting.", reason);
getContext().getLowMemoryDetector().logGCInfo(getConfiguration());
});
}

@Override
public void unableToMonitorLockNode(final Exception e) {
Halt.halt(1, () -> LOG.error("Lost ability to monitor Compactor lock, exiting.", e));
}
};
LockWatcher lw = new ServiceLockWatcher(Type.COMPACTOR, () -> false,
(type) -> getContext().getLowMemoryDetector().logGCInfo(getConfiguration()));

try {
for (int i = 0; i < 25; i++) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.cli.ConfigOpts;
import org.apache.accumulo.core.client.AccumuloClient;
import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
import org.apache.accumulo.core.clientImpl.thrift.TInfo;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
Expand All @@ -43,17 +44,15 @@
import org.apache.accumulo.core.gc.thrift.GCStatus;
import org.apache.accumulo.core.gc.thrift.GcCycleStats;
import org.apache.accumulo.core.lock.ServiceLock;
import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
import org.apache.accumulo.core.lock.ServiceLockData;
import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
import org.apache.accumulo.core.lock.ServiceLockSupport.HAServiceLockWatcher;
import org.apache.accumulo.core.metadata.AccumuloTable;
import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
import org.apache.accumulo.core.metrics.MetricsInfo;
import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
import org.apache.accumulo.core.spi.balancer.TableLoadBalancer;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.util.Halt;
import org.apache.accumulo.core.util.Timer;
import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
import org.apache.accumulo.core.util.threads.ThreadPools;
Expand Down Expand Up @@ -370,31 +369,32 @@ boolean moveToTrash(Path path) throws IOException {
private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedException {
var path = getContext().getServerPaths().createGarbageCollectorPath();

LockWatcher lockWatcher = new LockWatcher() {
@Override
public void lostLock(LockLossReason reason) {
Halt.halt("GC lock in zookeeper lost (reason = " + reason + "), exiting!", 1);
}
UUID zooLockUUID = UUID.randomUUID();
gcLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID);
HAServiceLockWatcher gcLockWatcher = new HAServiceLockWatcher(Type.GARBAGE_COLLECTOR);

@Override
public void unableToMonitorLockNode(final Exception e) {
// ACCUMULO-3651 Level changed to error and FATAL added to message for slf4j compatibility
Halt.halt(-1, () -> log.error("FATAL: No longer able to monitor lock node ", e));
while (true) {
gcLock.lock(gcLockWatcher, new ServiceLockData(zooLockUUID, addr.toString(), ThriftService.GC,
this.getResourceGroup()));

gcLockWatcher.waitForChange();

if (gcLockWatcher.isLockAcquired()) {
break;
}
};

UUID zooLockUUID = UUID.randomUUID();
gcLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID);
while (true) {
if (gcLock.tryLock(lockWatcher, new ServiceLockData(zooLockUUID, addr.toString(),
ThriftService.GC, this.getResourceGroup()))) {
log.debug("Got GC ZooKeeper lock");
return;
if (!gcLockWatcher.isFailedToAcquireLock()) {
throw new IllegalStateException("gc lock in unknown state");
}

gcLock.tryToCancelAsyncLockOrUnlock();

log.debug("Failed to get GC ZooKeeper lock, will retry");
sleepUninterruptibly(1, TimeUnit.SECONDS);
sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
}

log.info("Got GC lock.");

}

private HostAndPort startStatsService() {
Expand Down
Loading

0 comments on commit da51e06

Please sign in to comment.