Skip to content

Commit

Permalink
Merge branch '3.1' into remove-zoosession
Browse files Browse the repository at this point in the history
  • Loading branch information
ctubbsii committed Jan 10, 2025
2 parents 61a63e1 + e347e8b commit 84df5bd
Show file tree
Hide file tree
Showing 28 changed files with 604 additions and 371 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ static ContextClassLoaderFactory getContextFactory() {
}

// for testing
static synchronized void resetContextFactoryForTests() {
public static synchronized void resetContextFactoryForTests() {
FACTORY = null;
}

Expand Down
13 changes: 9 additions & 4 deletions core/src/main/java/org/apache/accumulo/core/conf/Property.java
Original file line number Diff line number Diff line change
Expand Up @@ -283,6 +283,9 @@ public enum Property {
+ " user-implementations of pluggable Accumulo features, such as the balancer"
+ " or volume chooser.",
"2.0.0"),
GENERAL_CACHE_MANAGER_IMPL("general.block.cache.manager.class",
TinyLfuBlockCacheManager.class.getName(), PropertyType.STRING,
"Specifies the class name of the block cache factory implementation.", "2.1.4"),
GENERAL_DELEGATION_TOKEN_LIFETIME("general.delegation.token.lifetime", "7d",
PropertyType.TIMEDURATION,
"The length of time that delegation tokens and secret keys are valid.", "1.7.0"),
Expand Down Expand Up @@ -514,6 +517,8 @@ public enum Property {
"Time to wait for clients to continue scans before closing a session.", "1.3.5"),
TSERV_DEFAULT_BLOCKSIZE("tserver.default.blocksize", "1M", PropertyType.BYTES,
"Specifies a default blocksize for the tserver caches.", "1.3.5"),
@Deprecated(since = "2.1.4")
@ReplacedBy(property = Property.GENERAL_CACHE_MANAGER_IMPL)
TSERV_CACHE_MANAGER_IMPL("tserver.cache.manager.class", TinyLfuBlockCacheManager.class.getName(),
PropertyType.STRING, "Specifies the class name of the block cache factory implementation.",
"2.0.0"),
Expand Down Expand Up @@ -1535,8 +1540,9 @@ public static boolean isValidTablePropertyKey(String key) {
RPC_MAX_MESSAGE_SIZE,

// block cache options
TSERV_CACHE_MANAGER_IMPL, TSERV_DATACACHE_SIZE, TSERV_INDEXCACHE_SIZE,
TSERV_SUMMARYCACHE_SIZE, SSERV_DATACACHE_SIZE, SSERV_INDEXCACHE_SIZE, SSERV_SUMMARYCACHE_SIZE,
GENERAL_CACHE_MANAGER_IMPL, TSERV_CACHE_MANAGER_IMPL, TSERV_DATACACHE_SIZE,
TSERV_INDEXCACHE_SIZE, TSERV_SUMMARYCACHE_SIZE, SSERV_DATACACHE_SIZE, SSERV_INDEXCACHE_SIZE,
SSERV_SUMMARYCACHE_SIZE,

// blocksize options
TSERV_DEFAULT_BLOCKSIZE, SSERV_DEFAULT_BLOCKSIZE,
Expand All @@ -1551,8 +1557,7 @@ public static boolean isValidTablePropertyKey(String key) {
COMPACTOR_MINTHREADS_TIMEOUT,

// others
TSERV_NATIVEMAP_ENABLED, TSERV_SCAN_MAX_OPENFILES, MANAGER_RECOVERY_WAL_EXISTENCE_CACHE_TIME,
TSERV_SESSION_MAXIDLE, TSERV_UPDATE_SESSION_MAXIDLE);
TSERV_NATIVEMAP_ENABLED, TSERV_SCAN_MAX_OPENFILES, MANAGER_RECOVERY_WAL_EXISTENCE_CACHE_TIME);

/**
* Checks if the given property may be changed via Zookeeper, but not recognized until the restart
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.commons.configuration2.MapConfiguration;
import org.apache.commons.configuration2.PropertiesConfiguration;
import org.apache.commons.configuration2.ex.ConfigurationException;
import org.apache.commons.configuration2.io.FileHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -211,8 +212,9 @@ private SiteConfiguration(Map<String,String> config) {
private static AbstractConfiguration getPropsFileConfig(URL accumuloPropsLocation) {
var config = new PropertiesConfiguration();
if (accumuloPropsLocation != null) {
var fileHandler = new FileHandler(config);
try (var reader = new InputStreamReader(accumuloPropsLocation.openStream(), UTF_8)) {
config.read(reader);
fileHandler.load(reader);
} catch (ConfigurationException | IOException e) {
throw new IllegalArgumentException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,10 @@ public class BlockCacheManagerFactory {
*/
public static synchronized BlockCacheManager getInstance(AccumuloConfiguration conf)
throws ReflectiveOperationException {
String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
@SuppressWarnings("deprecation")
var cacheManagerProp =
conf.resolve(Property.GENERAL_CACHE_MANAGER_IMPL, Property.TSERV_CACHE_MANAGER_IMPL);
String impl = conf.get(cacheManagerProp);
Class<? extends BlockCacheManager> clazz =
ClassLoaderUtil.loadClass(impl, BlockCacheManager.class);
LOG.info("Created new block cache manager of type: {}", clazz.getSimpleName());
Expand All @@ -55,7 +58,10 @@ public static synchronized BlockCacheManager getInstance(AccumuloConfiguration c
*/
public static synchronized BlockCacheManager getClientInstance(AccumuloConfiguration conf)
throws ReflectiveOperationException {
String impl = conf.get(Property.TSERV_CACHE_MANAGER_IMPL);
@SuppressWarnings("deprecation")
var cacheManagerProp =
conf.resolve(Property.GENERAL_CACHE_MANAGER_IMPL, Property.TSERV_CACHE_MANAGER_IMPL);
String impl = conf.get(cacheManagerProp);
Class<? extends BlockCacheManager> clazz =
Class.forName(impl).asSubclass(BlockCacheManager.class);
LOG.info("Created new block cache factory of type: {}", clazz.getSimpleName());
Expand Down
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.lock;

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

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.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 @@ -38,15 +38,15 @@ public class BlockCacheFactoryTest {
public void testCreateLruBlockCacheFactory() throws Exception {
DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManagerFactory.getInstance(cc);
}

@Test
public void testCreateTinyLfuBlockCacheFactory() throws Exception {
DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
BlockCacheManagerFactory.getInstance(cc);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ public class TestLruBlockCache {
@Test
public void testConfiguration() {
ConfigurationCopy cc = new ConfigurationCopy();
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(1019));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(1000023));
cc.set(Property.TSERV_DATACACHE_SIZE, Long.toString(1000027));
Expand Down Expand Up @@ -97,7 +97,7 @@ public void testBackgroundEvictionThread() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down Expand Up @@ -131,7 +131,7 @@ public void testCacheSimple() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down Expand Up @@ -189,7 +189,7 @@ public void testCacheEvictionSimple() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down Expand Up @@ -239,7 +239,7 @@ public void testCacheEvictionTwoPriorities() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down Expand Up @@ -307,7 +307,7 @@ public void testCacheEvictionThreePriorities() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down Expand Up @@ -432,7 +432,7 @@ public void testScanResistance() throws Exception {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
BlockCacheManager manager = BlockCacheManagerFactory.getInstance(cc);
cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(blockSize));
cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(maxSize));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ public void openReader(boolean cfsi, Range fence) throws IOException {

DefaultConfiguration dc = DefaultConfiguration.getInstance();
ConfigurationCopy cc = new ConfigurationCopy(dc);
cc.set(Property.TSERV_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
cc.set(Property.GENERAL_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
try {
manager = BlockCacheManagerFactory.getInstance(cc);
} catch (ReflectiveOperationException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1584,7 +1584,7 @@ private void runVersionTest(int version, ConfigurationCopy aconf) throws Excepti
byte[] data = baos.toByteArray();
SeekableByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
FSDataInputStream in2 = new FSDataInputStream(bais);
aconf.set(Property.TSERV_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
aconf.set(Property.GENERAL_CACHE_MANAGER_IMPL, TinyLfuBlockCacheManager.class.getName());
aconf.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(100000));
aconf.set(Property.TSERV_DATACACHE_SIZE, Long.toString(100000000));
aconf.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(100000000));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,7 @@ public class ZooInfoViewer implements KeywordExecutable {
DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC));
private static final Logger log = LoggerFactory.getLogger(ZooInfoViewer.class);

private final NullWatcher nullWatcher =
new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
private NullWatcher nullWatcher;

private static final String INDENT = " ";

Expand All @@ -103,6 +102,7 @@ public String description() {

@Override
public void execute(String[] args) throws Exception {
nullWatcher = new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));

ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
opts.parseArgs(ZooInfoViewer.class.getName(), args);
Expand All @@ -111,13 +111,15 @@ public void execute(String[] args) throws Exception {
log.info("print properties: {}", opts.printProps);
log.info("print instances: {}", opts.printInstanceIds);

var conf = opts.getSiteConfiguration();

try (ServerContext context = new ServerContext(conf)) {
try (ServerContext context = getContext(opts)) {
generateReport(context, opts);
}
}

ServerContext getContext(ZooInfoViewer.Opts opts) {
return new ServerContext(opts.getSiteConfiguration());
}

void generateReport(final ServerContext context, final ZooInfoViewer.Opts opts) throws Exception {

OutputStream outStream;
Expand Down
Loading

0 comments on commit 84df5bd

Please sign in to comment.