Skip to content

Commit

Permalink
Restore async refresh executor to auth caches
Browse files Browse the repository at this point in the history
Patch by Sam Tunnicliffe; reviewed by Brandon Williams and Benjamin
Lerer for CASSANDRA-15177
  • Loading branch information
beobal committed Mar 23, 2021
1 parent 1371883 commit d656f8a
Show file tree
Hide file tree
Showing 2 changed files with 5 additions and 2 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
4.0-beta5
* Execute background refreshing of auth caches on a dedicated executor (CASSANDRA-15177)
* Update bundled java and python drivers to 3.11.0 and 3.25.0 respectively (CASSANDRA-13951)
* Add io.netty.tryReflectionSetAccessible=true to j11 server options in order to enable netty to use Unsafe direct byte buffer construction (CASSANDRA-16493)
* Make cassandra-stress -node support host:port notation (CASSANDRA-16529)
Expand Down
6 changes: 4 additions & 2 deletions src/java/org/apache/cassandra/auth/AuthCache.java
Expand Up @@ -24,12 +24,12 @@
import java.util.function.IntConsumer;
import java.util.function.IntSupplier;

import com.google.common.util.concurrent.MoreExecutors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.LoadingCache;
import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
import org.apache.cassandra.utils.MBeanWrapper;

import static com.google.common.base.Preconditions.checkNotNull;
Expand All @@ -44,6 +44,7 @@ public class AuthCache<K, V> implements AuthCacheMBean
* Underlying cache. LoadingCache will call underlying load function on {@link #get} if key is not present
*/
protected volatile LoadingCache<K, V> cache;
private DebuggableThreadPoolExecutor cacheRefreshExecutor;

private String name;
private IntConsumer setValidityDelegate;
Expand Down Expand Up @@ -93,6 +94,7 @@ protected AuthCache(String name,
*/
protected void init()
{
this.cacheRefreshExecutor = new DebuggableThreadPoolExecutor(name + "Refresh", Thread.NORM_PRIORITY);
cache = initCache(null);
MBeanWrapper.instance.registerMBean(this, getObjectName());
}
Expand Down Expand Up @@ -218,7 +220,7 @@ protected LoadingCache<K, V> initCache(LoadingCache<K, V> existing)
.refreshAfterWrite(getUpdateInterval(), TimeUnit.MILLISECONDS)
.expireAfterWrite(getValidity(), TimeUnit.MILLISECONDS)
.maximumSize(getMaxEntries())
.executor(MoreExecutors.directExecutor())
.executor(cacheRefreshExecutor)
.build(loadFunction::apply);
}

Expand Down

0 comments on commit d656f8a

Please sign in to comment.