Skip to content

Commit

Permalink
Made lock acquisition consistent
Browse files Browse the repository at this point in the history
Modified SimpleGarbageCollector to acquire the service
lock in the same manner that other HA services acquire
the service lock.

Created a new class called ServiceLockSupport that is
currently used to hold LockWatcher implementations for
HA and non-HA servers.

Closes apache#4839
  • Loading branch information
dlmarion committed Jan 6, 2025
1 parent 9ac9ddd commit 50c0c06
Show file tree
Hide file tree
Showing 9 changed files with 192 additions and 294 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* 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.function.Consumer;
import java.util.function.Supplier;

import org.apache.accumulo.core.fate.zookeeper.ServiceLock.AccumuloLockWatcher;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
import org.apache.accumulo.core.util.Halt;
import org.apache.zookeeper.KeeperException.NoAuthException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class ServiceLockSupport {

/**
* 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 String serviceName;
private volatile boolean acquiredLock = false;
private volatile boolean failedToAcquireLock = false;

public HAServiceLockWatcher(String serviceName) {
this.serviceName = serviceName;
}

@Override
public void lostLock(LockLossReason reason) {
Halt.halt(serviceName + " 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", serviceName, e));

}

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

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", serviceName, e);

if (e instanceof NoAuthException) {
String msg =
"Failed to acquire " + serviceName + " 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", serviceName);
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 String serviceName;
private final Supplier<Boolean> shuttingDown;
private final Consumer<String> lostLockAction;

public ServiceLockWatcher(String serviceName, Supplier<Boolean> shuttingDown,
Consumer<String> lostLockAction) {
this.serviceName = serviceName;
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.", serviceName, reason);
}
lostLockAction.accept(serviceName);
});
}

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

}

}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ServiceLockSupport.HAServiceLockWatcher;
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.metadata.TServerInstance;
import org.apache.accumulo.core.metadata.schema.Ample;
Expand Down Expand Up @@ -223,11 +224,11 @@ protected void getCoordinatorLock(HostAndPort clientAddress)
ServiceLock.path(lockPath), zooLockUUID);
while (true) {

CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
HAServiceLockWatcher coordinatorLockWatcher = new HAServiceLockWatcher("coordinator");
coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes(UTF_8));

coordinatorLockWatcher.waitForChange();
if (coordinatorLockWatcher.isAcquiredLock()) {
if (coordinatorLockWatcher.isLockAcquired()) {
break;
}
if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -66,8 +66,8 @@
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
import org.apache.accumulo.core.fate.zookeeper.ServiceLockSupport.ServiceLockWatcher;
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
import org.apache.accumulo.core.file.FileOperations;
Expand All @@ -91,7 +91,6 @@
import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.trace.thrift.TInfo;
import org.apache.accumulo.core.util.Halt;
import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.ServerServices;
import org.apache.accumulo.core.util.ServerServices.Service;
Expand Down Expand Up @@ -288,20 +287,9 @@ protected void announceExistence(HostAndPort clientAddress)

compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
ServiceLock.path(zPath), compactorId);
LockWatcher lw = new LockWatcher() {
@Override
public void lostLock(final LockLossReason reason) {
Halt.halt(1, () -> {
LOG.error("Compactor lost lock (reason = {}), exiting.", reason);
gcLogger.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("compactor", () -> false,
(name) -> gcLogger.logGCInfo(getConfiguration()));

try {
byte[] lockContent =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,7 @@
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
import org.apache.accumulo.core.fate.zookeeper.ServiceLockSupport.HAServiceLockWatcher;
import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
import org.apache.accumulo.core.gc.thrift.GCStatus;
import org.apache.accumulo.core.gc.thrift.GcCycleStats;
Expand All @@ -44,7 +43,6 @@
import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.trace.thrift.TInfo;
import org.apache.accumulo.core.util.Halt;
import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.core.util.ServerServices;
import org.apache.accumulo.core.util.ServerServices.Service;
Expand Down Expand Up @@ -365,31 +363,31 @@ boolean moveToTrash(Path path) throws IOException {
private void getZooLock(HostAndPort addr) throws KeeperException, InterruptedException {
var path = ServiceLock.path(getContext().getZooKeeperRoot() + Constants.ZGC_LOCK);

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);

@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) {
HAServiceLockWatcher gcLockWatcher = new HAServiceLockWatcher("gc");
gcLock.lock(gcLockWatcher,
new ServerServices(addr.toString(), Service.GC_CLIENT).toString().getBytes(UTF_8));

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 ServerServices(addr.toString(), Service.GC_CLIENT).toString().getBytes(UTF_8))) {
log.debug("Got GC ZooKeeper lock");
return;
if (!gcLockWatcher.isFailedToAcquireLock()) {
throw new IllegalStateException("gc lock in unknown state");
}
log.debug("Failed to get GC ZooKeeper lock, will retry");
sleepUninterruptibly(1, TimeUnit.SECONDS);

gcLock.tryToCancelAsyncLockOrUnlock();

sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
}

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

}

private HostAndPort startStatsService() {
Expand Down
Loading

0 comments on commit 50c0c06

Please sign in to comment.