Skip to content

Commit

Permalink
Test EvictableCache load counts with different cache bounds
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Oct 11, 2022
1 parent e149647 commit bdd3132
Show file tree
Hide file tree
Showing 2 changed files with 132 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -75,17 +75,24 @@ public void testLoad()
public void testEvictBySize()
throws Exception
{
int maximumSize = 10;
Cache<Integer, Integer> cache = EvictableCacheBuilder.newBuilder()
.maximumSize(10)
.maximumSize(maximumSize)
.build();

for (int i = 0; i < 10_000; i++) {
int value = i * 10;
assertEquals((Object) cache.get(i, () -> value), value);
}
cache.cleanUp();
assertEquals(cache.size(), 10);
assertEquals(((EvictableCache<?, ?>) cache).tokensCount(), 10);
assertEquals(cache.size(), maximumSize);
assertEquals(((EvictableCache<?, ?>) cache).tokensCount(), maximumSize);

// Ensure cache is effective, i.e. some entries preserved
int lastKey = 10_000 - 1;
assertEquals((Object) cache.get(lastKey, () -> {
throw new UnsupportedOperationException();
}), lastKey * 10);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
Expand All @@ -109,6 +116,12 @@ public void testEvictByWeight()
assertThat(cache.asMap().keySet().stream().mapToInt(i -> i).sum()).as("key sum").isLessThanOrEqualTo(20);
assertThat(cache.asMap().values()).as("values").hasSize(cacheSize);
assertThat(cache.asMap().values().stream().mapToInt(String::length).sum()).as("values length sum").isLessThanOrEqualTo(20);

// Ensure cache is effective, i.e. some entries preserved
int lastKey = 10 - 1;
assertEquals(cache.get(lastKey, () -> {
throw new UnsupportedOperationException();
}), Strings.repeat("a", lastKey));
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
Expand All @@ -135,6 +148,45 @@ public void testEvictByTime()
assertThat(cache.asMap().values()).as("values").hasSize(cacheSize);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testPreserveValueLoadedAfterTimeExpiration()
throws Exception
{
TestingTicker ticker = new TestingTicker();
int ttl = 100;
Cache<Integer, String> cache = EvictableCacheBuilder.newBuilder()
.ticker(ticker)
.expireAfterWrite(ttl, TimeUnit.MILLISECONDS)
.build();
int key = 11;

assertEquals(cache.get(key, () -> "11 ala ma kota"), "11 ala ma kota");
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// Should be served from the cache
assertEquals(cache.get(key, () -> "something else"), "11 ala ma kota");
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

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

// 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
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(((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);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testReplace()
throws Exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,26 +81,41 @@ public void testLoad()
public void testEvictBySize()
throws Exception
{
int maximumSize = 10;
AtomicInteger loads = new AtomicInteger();
LoadingCache<Integer, String> cache = EvictableCacheBuilder.newBuilder()
.maximumSize(10)
.build(CacheLoader.from(key -> "abc" + key));
.maximumSize(maximumSize)
.build(CacheLoader.from(key -> {
loads.incrementAndGet();
return "abc" + key;
}));

for (int i = 0; i < 10_000; i++) {
assertEquals((Object) cache.get(i), "abc" + i);
}
cache.cleanUp();
assertEquals(cache.size(), 10);
assertEquals(((EvictableCache<?, ?>) cache).tokensCount(), 10);
assertEquals(cache.size(), maximumSize);
assertEquals(((EvictableCache<?, ?>) cache).tokensCount(), maximumSize);
assertEquals(loads.get(), 10_000);

// Ensure cache is effective, i.e. no new load
int lastKey = 10_000 - 1;
assertEquals((Object) cache.get(lastKey), "abc" + lastKey);
assertEquals(loads.get(), 10_000);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testEvictByWeight()
throws Exception
{
AtomicInteger loads = new AtomicInteger();
LoadingCache<Integer, String> cache = EvictableCacheBuilder.newBuilder()
.maximumWeight(20)
.weigher((Integer key, String value) -> value.length())
.build(CacheLoader.from(key -> Strings.repeat("a", key)));
.build(CacheLoader.from(key -> {
loads.incrementAndGet();
return Strings.repeat("a", key);
}));

for (int i = 0; i < 10; i++) {
assertEquals((Object) cache.get(i), Strings.repeat("a", i));
Expand All @@ -113,17 +128,27 @@ public void testEvictByWeight()
assertThat(cache.asMap().keySet().stream().mapToInt(i -> i).sum()).as("key sum").isLessThanOrEqualTo(20);
assertThat(cache.asMap().values()).as("values").hasSize(cacheSize);
assertThat(cache.asMap().values().stream().mapToInt(String::length).sum()).as("values length sum").isLessThanOrEqualTo(20);
assertEquals(loads.get(), 10);

// Ensure cache is effective, i.e. no new load
int lastKey = 10 - 1;
assertEquals((Object) cache.get(lastKey), Strings.repeat("a", lastKey));
assertEquals(loads.get(), 10);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testEvictByTime()
{
TestingTicker ticker = new TestingTicker();
int ttl = 100;
AtomicInteger loads = new AtomicInteger();
LoadingCache<Integer, String> cache = EvictableCacheBuilder.newBuilder()
.ticker(ticker)
.expireAfterWrite(ttl, TimeUnit.MILLISECONDS)
.build(CacheLoader.from(k -> k + " ala ma kota"));
.build(CacheLoader.from(k -> {
loads.incrementAndGet();
return k + " ala ma kota";
}));

assertEquals(cache.getUnchecked(1), "1 ala ma kota");
ticker.increment(ttl, MILLISECONDS);
Expand All @@ -136,6 +161,52 @@ public void testEvictByTime()
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(cacheSize);
assertThat(cache.asMap().keySet()).as("keySet").hasSize(cacheSize);
assertThat(cache.asMap().values()).as("values").hasSize(cacheSize);
assertEquals(loads.get(), 2);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testPreserveValueLoadedAfterTimeExpiration()
{
TestingTicker ticker = new TestingTicker();
int ttl = 100;
AtomicInteger loads = new AtomicInteger();
LoadingCache<Integer, String> cache = EvictableCacheBuilder.newBuilder()
.ticker(ticker)
.expireAfterWrite(ttl, TimeUnit.MILLISECONDS)
.build(CacheLoader.from(k -> {
loads.incrementAndGet();
return k + " ala ma kota";
}));
int key = 11;

assertEquals(cache.getUnchecked(key), "11 ala ma kota");
assertThat(loads.get()).as("initial load count").isEqualTo(1);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

// Should be served from the cache
assertEquals(cache.getUnchecked(key), "11 ala ma kota");
assertThat(loads.get()).as("loads count should not change before value expires").isEqualTo(1);
assertThat(((EvictableCache<?, ?>) cache).tokensCount()).as("tokensCount").isEqualTo(1);

ticker.increment(ttl, MILLISECONDS);
// 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);

// 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(((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(((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);
}

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

0 comments on commit bdd3132

Please sign in to comment.