Skip to content

Commit

Permalink
Always test all caches in TestCachingJdbcClient
Browse files Browse the repository at this point in the history
Ensure the action under test does not unexpectedly hit other caches.
This is especially useful when testing than a cache was not hit
at all. 0 hits could be a result of asserting on a wrong cache.
  • Loading branch information
findepi committed Jul 21, 2022
1 parent 1383ea9 commit 0dc7113
Showing 1 changed file with 157 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,11 @@
*/
package io.trino.plugin.jdbc;

import com.google.common.cache.CacheStats;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
import io.airlift.units.Duration;
import io.trino.collect.cache.CacheStatsAssertions;
import io.trino.plugin.base.session.SessionPropertiesProvider;
import io.trino.plugin.jdbc.credential.ExtraCredentialConfig;
import io.trino.spi.connector.ColumnMetadata;
Expand All @@ -37,18 +37,24 @@

import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
import java.util.stream.Stream;

import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.collect.cache.CacheStatsAssertions.assertCacheStats;
import static io.trino.plugin.jdbc.TestCachingJdbcClient.CachingJdbcCache.STATISTICS_CACHE;
import static io.trino.spi.session.PropertyMetadata.stringProperty;
import static io.trino.spi.testing.InterfaceTestUtils.assertAllMethodsOverridden;
import static io.trino.spi.type.IntegerType.INTEGER;
Expand All @@ -58,10 +64,12 @@
import static java.lang.Math.min;
import static java.lang.String.format;
import static java.util.Collections.emptyList;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.TimeUnit.DAYS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.function.Function.identity;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

Expand Down Expand Up @@ -174,6 +182,11 @@ public void testTableHandleOfQueryCached()
createTable(phantomTable);
PreparedQuery query = new PreparedQuery(format("SELECT * FROM %s.phantom_table", schema), ImmutableList.of());
JdbcTableHandle cachedTable = cachingJdbcClient.getTableHandle(SESSION, query);
assertCacheStats(cachingJdbcClient)
// cache is not used, as the table handle has the columns list embedded
.afterRunning(() -> {
cachingJdbcClient.getColumns(SESSION, cachedTable);
});
dropTable(phantomTable);

assertThatThrownBy(() -> jdbcClient.getTableHandle(SESSION, query))
Expand Down Expand Up @@ -761,24 +774,157 @@ public void testEverythingImplemented()
assertAllMethodsOverridden(JdbcClient.class, CachingJdbcClient.class);
}

private static CacheStatsAssertions assertTableNamesCache(CachingJdbcClient cachingJdbcClient)
private static String randomSuffix()
{
return assertCacheStats(cachingJdbcClient::getTableNamesCacheStats);
String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX);
return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length()));
}

private static CacheStatsAssertions assertColumnCacheStats(CachingJdbcClient client)
private static SingleJdbcCacheStatsAssertions assertTableNamesCache(CachingJdbcClient client)
{
return assertCacheStats(client::getColumnsCacheStats);
return assertCacheStats(client, CachingJdbcCache.TABLE_NAMES_CACHE);
}

private static CacheStatsAssertions assertStatisticsCacheStats(CachingJdbcClient client)
private static SingleJdbcCacheStatsAssertions assertColumnCacheStats(CachingJdbcClient client)
{
return assertCacheStats(client::getStatisticsCacheStats);
return assertCacheStats(client, CachingJdbcCache.COLUMNS_CACHE);
}

private static String randomSuffix()
private static SingleJdbcCacheStatsAssertions assertStatisticsCacheStats(CachingJdbcClient client)
{
String randomSuffix = Long.toString(abs(random.nextLong()), MAX_RADIX);
return randomSuffix.substring(0, min(RANDOM_SUFFIX_LENGTH, randomSuffix.length()));
return assertCacheStats(client, STATISTICS_CACHE);
}

private static SingleJdbcCacheStatsAssertions assertCacheStats(CachingJdbcClient client, CachingJdbcCache cache)
{
return new SingleJdbcCacheStatsAssertions(client, cache);
}

private static JdbcCacheStatsAssertions assertCacheStats(CachingJdbcClient client)
{
return new JdbcCacheStatsAssertions(client);
}

private static class SingleJdbcCacheStatsAssertions
{
private CachingJdbcCache chosenCache;
private JdbcCacheStatsAssertions delegate;

private SingleJdbcCacheStatsAssertions(CachingJdbcClient jdbcClient, CachingJdbcCache chosenCache)
{
this.chosenCache = requireNonNull(chosenCache, "chosenCache is null");
delegate = new JdbcCacheStatsAssertions(jdbcClient);
}

public SingleJdbcCacheStatsAssertions loads(long value)
{
delegate.loads(chosenCache, value);
return this;
}

public SingleJdbcCacheStatsAssertions hits(long value)
{
delegate.hits(chosenCache, value);
return this;
}

public SingleJdbcCacheStatsAssertions misses(long value)
{
delegate.misses(chosenCache, value);
return this;
}

public void afterRunning(Runnable runnable)
{
delegate.afterRunning(runnable);
}

public <T> T calling(Callable<T> callable)
throws Exception
{
return delegate.calling(callable);
}
}

private static class JdbcCacheStatsAssertions
{
private final CachingJdbcClient jdbcClient;

private final Map<CachingJdbcCache, Long> loads = new HashMap<>();
private final Map<CachingJdbcCache, Long> hits = new HashMap<>();
private final Map<CachingJdbcCache, Long> misses = new HashMap<>();

public JdbcCacheStatsAssertions(CachingJdbcClient jdbcClient)
{
this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null");
}

public JdbcCacheStatsAssertions loads(CachingJdbcCache cache, long value)
{
loads.put(cache, value);
return this;
}

public JdbcCacheStatsAssertions hits(CachingJdbcCache cache, long value)
{
hits.put(cache, value);
return this;
}

public JdbcCacheStatsAssertions misses(CachingJdbcCache cache, long value)
{
misses.put(cache, value);
return this;
}

public void afterRunning(Runnable runnable)
{
try {
calling(() -> {
runnable.run();
return null;
});
}
catch (Exception e) {
throw new RuntimeException(e);
}
}

public <T> T calling(Callable<T> callable)
throws Exception
{
Map<CachingJdbcCache, CacheStats> beforeStats = Stream.of(CachingJdbcCache.values())
.collect(toImmutableMap(identity(), cache -> cache.statsGetter.apply(jdbcClient)));
T value = callable.call();
Map<CachingJdbcCache, CacheStats> afterStats = Stream.of(CachingJdbcCache.values())
.collect(toImmutableMap(identity(), cache -> cache.statsGetter.apply(jdbcClient)));

for (CachingJdbcCache cache : CachingJdbcCache.values()) {
long loadDelta = afterStats.get(cache).loadCount() - beforeStats.get(cache).loadCount();
long missesDelta = afterStats.get(cache).missCount() - beforeStats.get(cache).missCount();
long hitsDelta = afterStats.get(cache).hitCount() - beforeStats.get(cache).hitCount();

assertThat(loadDelta).as(cache + " loads (delta)").isEqualTo(loads.getOrDefault(cache, 0L));
assertThat(hitsDelta).as(cache + " hits (delta)").isEqualTo(hits.getOrDefault(cache, 0L));
assertThat(missesDelta).as(cache + " misses (delta)").isEqualTo(misses.getOrDefault(cache, 0L));
}

return value;
}
}

enum CachingJdbcCache
{
TABLE_NAMES_CACHE(CachingJdbcClient::getTableNamesCacheStats),
COLUMNS_CACHE(CachingJdbcClient::getColumnsCacheStats),
STATISTICS_CACHE(CachingJdbcClient::getStatisticsCacheStats),
/**/;

private final Function<CachingJdbcClient, CacheStats> statsGetter;

CachingJdbcCache(Function<CachingJdbcClient, CacheStats> statsGetter)
{
this.statsGetter = requireNonNull(statsGetter, "statsGetter is null");
}
}
}

0 comments on commit 0dc7113

Please sign in to comment.