Skip to content

Commit

Permalink
Load value once after expiration in EvictableCache
Browse files Browse the repository at this point in the history
Before the change, when value was time-expired, the new token inserted
for the cache loading was immediately discarded due to expiration event.

Co-authored-by: Anton Tanasenko <[email protected]>
  • Loading branch information
findepi and atanasenko committed Oct 15, 2022
1 parent fe5482b commit 17faae3
Show file tree
Hide file tree
Showing 3 changed files with 41 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Verify.verify;
Expand Down Expand Up @@ -66,6 +67,8 @@ class EvictableCache<K, V>
// The dataCache must be bounded.
private final LoadingCache<Token<K>, V> dataCache;

private final AtomicInteger invalidations = new AtomicInteger();

EvictableCache(CacheBuilder<? super Token<K>, ? super V> cacheBuilder, CacheLoader<? super K, V> cacheLoader)
{
dataCache = buildUnsafeCache(
Expand Down Expand Up @@ -103,9 +106,22 @@ public V get(K key, Callable<? extends V> valueLoader)
throws ExecutionException
{
Token<K> newToken = new Token<>(key);
int invalidations = this.invalidations.get();
Token<K> token = tokens.computeIfAbsent(key, ignored -> newToken);
try {
return dataCache.get(token, valueLoader);
V value = dataCache.get(token, valueLoader);
if (invalidations == this.invalidations.get()) {
// Revive token if it got expired before reloading
if (tokens.putIfAbsent(key, token) == null) {
// Revived
if (!dataCache.asMap().containsKey(token)) {
// We revived, but the token does not correspond to a live entry anymore.
// It would stay in tokens forever, so let's remove it.
tokens.remove(key, token);
}
}
}
return value;
}
catch (Throwable e) {
if (newToken == token) {
Expand All @@ -123,9 +139,22 @@ public V get(K key)
throws ExecutionException
{
Token<K> newToken = new Token<>(key);
int invalidations = this.invalidations.get();
Token<K> token = tokens.computeIfAbsent(key, ignored -> newToken);
try {
return dataCache.get(token);
V value = dataCache.get(token);
if (invalidations == this.invalidations.get()) {
// Revive token if it got expired before reloading
if (tokens.putIfAbsent(key, token) == null) {
// Revived
if (!dataCache.asMap().containsKey(token)) {
// We revived, but the token does not correspond to a live entry anymore.
// It would stay in tokens forever, so let's remove it.
tokens.remove(key, token);
}
}
}
return value;
}
catch (Throwable e) {
if (newToken == token) {
Expand Down Expand Up @@ -219,6 +248,7 @@ int tokensCount()
@Override
public void invalidate(Object key)
{
invalidations.incrementAndGet();
@SuppressWarnings("SuspiciousMethodCalls") // Object passed to map as key K
Token<K> token = tokens.remove(key);
if (token != null) {
Expand All @@ -229,6 +259,7 @@ public void invalidate(Object key)
@Override
public void invalidateAll()
{
invalidations.incrementAndGet();
dataCache.invalidateAll();
tokens.clear();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -170,21 +170,16 @@ public void testPreserveValueLoadedAfterTimeExpiration()
ticker.increment(ttl, MILLISECONDS);
// Should be reloaded
assertEquals(cache.get(key, () -> "new value"), "new value");
// TODO (https://github.com/trinodb/trino/issues/14545) tokensCount should be 1; 0 means we lost the token for a live entry
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(0);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// Should be served from the cache
// TODO (https://github.com/trinodb/trino/issues/14545) this should return "new value" inserted into the cache above; it's not doing that due to the token being lost
assertEquals(cache.get(key, () -> "something yet different"), "something yet different");
// TODO (https://github.com/trinodb/trino/issues/14545) loads count should be 2; it got incremented because we lost the token for a live entry
assertEquals(cache.get(key, () -> "something yet different"), "new value");
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// TODO (https://github.com/trinodb/trino/issues/14545) cache size should be 1; it is misreported, because there are two live entries for the same key, due to first token being lost
assertThat(cache.size()).as("cacheSize").isEqualTo(2);
assertThat(cache.size()).as("cacheSize").isEqualTo(1);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);
assertThat(cache.asMap().keySet()).as("keySet").hasSize(1);
// TODO (https://github.com/trinodb/trino/issues/14545) values size should be 1; it is misreported, because there are two live entries for the same key, due to first token being lost
assertThat(cache.asMap().values()).as("values").hasSize(2);
assertThat(cache.asMap().values()).as("values").hasSize(1);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,21 +192,17 @@ public void testPreserveValueLoadedAfterTimeExpiration()
// Should be reloaded
assertEquals(cache.getUnchecked(key), "11 ala ma kota");
assertThat(loads.get()).as("loads count should reflect reloading of value after expiration").isEqualTo(2);
// TODO (https://github.com/trinodb/trino/issues/14545) tokensCount should be 1; 0 means we lost the token for a live entry
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(0);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// Should be served from the cache
assertEquals(cache.getUnchecked(key), "11 ala ma kota");
// TODO (https://github.com/trinodb/trino/issues/14545) loads count should be 2; it got incremented because we lost the token for a live entry
assertThat(loads.get()).as("loads count should not change before value expires again").isEqualTo(3);
assertThat(loads.get()).as("loads count should not change before value expires again").isEqualTo(2);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// TODO (https://github.com/trinodb/trino/issues/14545) cache size should be 1; it is misreported, because there are two live entries for the same key, due to first token being lost
assertThat(cache.size()).as("cacheSize").isEqualTo(2);
assertThat(cache.size()).as("cacheSize").isEqualTo(1);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);
assertThat(cache.asMap().keySet()).as("keySet").hasSize(1);
// TODO (https://github.com/trinodb/trino/issues/14545) values size should be 1; it is misreported, because there are two live entries for the same key, due to first token being lost
assertThat(cache.asMap().values()).as("values").hasSize(2);
assertThat(cache.asMap().values()).as("values").hasSize(1);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS, dataProvider = "testDisabledCacheDataProvider")
Expand Down

0 comments on commit 17faae3

Please sign in to comment.