Skip to content

Commit

Permalink
avoid reading from zookeeper when clearing zoocache entries
Browse files Browse the repository at this point in the history
This change indirectly fixes ZombieScanIT.  That test was creating
scanners running in thread and interrupting those threads.  The
interrupts were routinely being ingnored by code in zoocache that
ignores interrupts when reading from zookeeper.  Looking at the code
that was reading from zookeeper, that code was attempting clear entries
in the cache.  Modified this code to avoid reading from zookeeper when
clearing the cache.
  • Loading branch information
keith-turner committed Oct 22, 2024
1 parent 9416715 commit 9e11aae
Show file tree
Hide file tree
Showing 3 changed files with 16 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,11 +59,6 @@ public boolean isLockHeld(String server, String session) {

@Override
public void invalidateCache(String tserver) {
var hostAndPort = HostAndPort.fromString(tserver);
ctx.getServerPaths().getTabletServer(rg -> true, AddressSelector.exact(hostAndPort), false)
.forEach(slp -> {
ctx.getZooCache().clear(slp.toString());
});
ctx.getZooCache().clear(path -> path.contains(tserver));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import java.util.concurrent.locks.LockSupport;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Predicate;

import org.apache.accumulo.core.lock.ServiceLock;
import org.apache.accumulo.core.lock.ServiceLockData;
Expand Down Expand Up @@ -513,24 +514,31 @@ boolean childrenCached(String zPath) {
}

/**
* Clears this cache of all information about nodes rooted at the given path.
*
* @param zPath path of top node
* Removes all paths in the cache match the predicate.
*/
public void clear(String zPath) {
public void clear(Predicate<String> pathPredicate) {
Preconditions.checkState(!closed);
cacheWriteLock.lock();
try {
cache.keySet().removeIf(path -> path.startsWith(zPath));
childrenCache.keySet().removeIf(path -> path.startsWith(zPath));
statCache.keySet().removeIf(path -> path.startsWith(zPath));
cache.keySet().removeIf(pathPredicate);
childrenCache.keySet().removeIf(pathPredicate);
statCache.keySet().removeIf(pathPredicate);

immutableCache = new ImmutableCacheCopies(++updateCount, cache, statCache, childrenCache);
} finally {
cacheWriteLock.unlock();
}
}

/**
* Clears this cache of all information about nodes rooted at the given path.
*
* @param zPath path of top node
*/
public void clear(String zPath) {
clear(path -> path.startsWith(zPath));
}

public Optional<ServiceLockData> getLockData(ServiceLockPath path) {
List<String> children = ServiceLock.validateAndSort(path, getChildren(path.toString()));
if (children == null || children.isEmpty()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,10 @@
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.replay;
import static org.easymock.EasyMock.verify;

import java.util.List;

import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
import org.apache.accumulo.core.lock.ServiceLockPaths;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

public class ZookeeperLockCheckerTest {
private ClientContext context;
Expand All @@ -46,19 +41,4 @@ public void setUp() {
replay(context);
zklc = new ZookeeperLockChecker(context);
}

@Test
public void testInvalidateCache() {
expect(zc.getChildren(context.getZooKeeperRoot())).andReturn(List.of(Constants.ZTSERVERS))
.anyTimes();
expect(zc.getChildren(context.getZooKeeperRoot() + Constants.ZTSERVERS))
.andReturn(List.of(Constants.DEFAULT_RESOURCE_GROUP_NAME)).anyTimes();
expect(zc.get(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/"
+ Constants.DEFAULT_RESOURCE_GROUP_NAME + "/server")).andReturn(new byte[0]).anyTimes();
zc.clear(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/"
+ Constants.DEFAULT_RESOURCE_GROUP_NAME + "/server");
replay(zc);
zklc.invalidateCache("server");
verify(zc);
}
}

0 comments on commit 9e11aae

Please sign in to comment.