Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ class EvictableCache<K, V>
tokens.remove(token.getKey(), token);
}
}),
new TokenCacheLoader<>(cacheLoader));
new TokenCacheLoader<>(cacheLoader, tokens));
}

@SuppressModernizer // CacheBuilder.build(CacheLoader) is forbidden, advising to use this class as a safety-adding wrapper.
Expand All @@ -104,8 +104,13 @@ public V get(K key, Callable<? extends V> valueLoader)
{
Token<K> newToken = new Token<>(key);
Token<K> token = tokens.computeIfAbsent(key, ignored -> newToken);
Callable<? extends V> valueLoaderImpl = () -> {
// revive token if it got expired before reloading
tokens.computeIfAbsent(token.getKey(), ignored -> token);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

would this be equivalent?

Suggested change
tokens.computeIfAbsent(token.getKey(), ignored -> token);
tokens.computeIfAbsent(key, ignored -> token);

return valueLoader.call();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just thinking -- should the revival happen before or after valueLoader.call?
or it doesn't matter?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

related -- if valueLoader throws and this wasn't a new token, we can revive it and leak

};
try {
return dataCache.get(token, valueLoader);
return dataCache.get(token, valueLoaderImpl);
}
catch (Throwable e) {
if (newToken == token) {
Expand Down Expand Up @@ -394,16 +399,20 @@ private static class TokenCacheLoader<K, V>
extends CacheLoader<Token<K>, V>
{
private final CacheLoader<? super K, V> delegate;
private final ConcurrentHashMap<K, Token<K>> tokens;

public TokenCacheLoader(CacheLoader<? super K, V> delegate)
public TokenCacheLoader(CacheLoader<? super K, V> delegate, ConcurrentHashMap<K, Token<K>> tokens)
{
this.delegate = requireNonNull(delegate, "delegate is null");
this.tokens = requireNonNull(tokens, "tokens is null");
}

@Override
public V load(Token<K> token)
throws Exception
{
// revive token if it got expired before reloading
tokens.computeIfAbsent(token.getKey(), ignored -> token);
return delegate.load(token.getKey());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
Expand All @@ -43,8 +44,10 @@
import static io.trino.collect.cache.CacheStatsAssertions.assertCacheStats;
import static java.lang.Math.toIntExact;
import static java.lang.String.format;
import static java.time.temporal.ChronoUnit.MILLIS;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.concurrent.TimeUnit.DAYS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
Expand Down Expand Up @@ -108,6 +111,26 @@ public void testEvictByWeight()
assertThat(cache.asMap().values().stream().mapToInt(String::length).sum()).as("values length sum").isLessThanOrEqualTo(20);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testLoadOnceWithTimeEviction()
throws Exception
{
int ttl = 50;
int expectedCalls = 10;

AtomicInteger counter = new AtomicInteger();
Cache<String, Integer> cache = EvictableCacheBuilder.newBuilder()
.expireAfterWrite(ttl, MILLISECONDS)
.build();

Instant until = Instant.now().plus(ttl * expectedCalls, MILLIS);
while (until.isAfter(Instant.now())) {
Comment on lines 126 to 127
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use Stopwatch -- it's ligheter-weight and easier to lead.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW i wanted to put a PR with a test preventing memory leak (#14476 (comment)) and end-up also writing my version of this test #14502. not sure if this helps

cache.get("foo", counter::incrementAndGet);
Thread.sleep(1);
}
assertThat(counter.get()).isEqualTo(expectedCalls);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the timeout until.isAfter(Instant.now()) condition is checked before a call is made, so we should expect allow one more call here

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It never goes past the threshold of the last expiration. Or am I missing something?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You don't know how much time elapses between checking the clock and invoking cache.get

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

am i missing something?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i think this boils down to the fact it's much better to use ~Ticker for tests, per @losipiuk #14502 (comment)

}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testReplace()
throws Exception
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
Expand All @@ -47,7 +48,9 @@
import static io.trino.collect.cache.CacheStatsAssertions.assertCacheStats;
import static java.lang.Math.toIntExact;
import static java.lang.String.format;
import static java.time.temporal.ChronoUnit.MILLIS;
import static java.util.concurrent.Executors.newFixedThreadPool;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
Expand Down Expand Up @@ -112,6 +115,29 @@ public void testEvictByWeight()
assertThat(cache.asMap().values().stream().mapToInt(String::length).sum()).as("values length sum").isLessThanOrEqualTo(20);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS)
public void testLoadOnceWithTimeEviction()
throws Exception
{
int ttl = 50;
int expectedCalls = 10;

AtomicInteger counter = new AtomicInteger();
LoadingCache<String, String> cache = EvictableCacheBuilder.newBuilder()
.expireAfterWrite(ttl, MILLISECONDS)
.build(CacheLoader.from(k -> {
counter.incrementAndGet();
return k;
}));

Instant until = Instant.now().plus(ttl * expectedCalls, MILLIS);
while (until.isAfter(Instant.now())) {
cache.get("foo");
Thread.sleep(1);
}
assertThat(counter.get()).isEqualTo(expectedCalls);
}

@Test(timeOut = TEST_TIMEOUT_MILLIS, dataProvider = "testDisabledCacheDataProvider")
public void testDisabledCache(String behavior)
throws Exception
Expand Down