HDDS-1861. Fix TableCacheImpl cleanup logic. (#1165)

This commit is contained in:
Bharat Viswanadham 2019-07-25 22:30:06 -07:00 committed by GitHub
parent ce99cc31e9
commit 3426777140
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

View File

@ -21,8 +21,9 @@
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import java.util.TreeSet; import java.util.NavigableSet;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;
@ -47,7 +48,7 @@ public class TableCacheImpl<CACHEKEY extends CacheKey,
CACHEVALUE extends CacheValue> implements TableCache<CACHEKEY, CACHEVALUE> { CACHEVALUE extends CacheValue> implements TableCache<CACHEKEY, CACHEVALUE> {
private final ConcurrentHashMap<CACHEKEY, CACHEVALUE> cache; private final ConcurrentHashMap<CACHEKEY, CACHEVALUE> cache;
private final TreeSet<EpochEntry<CACHEKEY>> epochEntries; private final NavigableSet<EpochEntry<CACHEKEY>> epochEntries;
private ExecutorService executorService; private ExecutorService executorService;
private CacheCleanupPolicy cleanupPolicy; private CacheCleanupPolicy cleanupPolicy;
@ -55,7 +56,7 @@ public class TableCacheImpl<CACHEKEY extends CacheKey,
public TableCacheImpl(CacheCleanupPolicy cleanupPolicy) { public TableCacheImpl(CacheCleanupPolicy cleanupPolicy) {
cache = new ConcurrentHashMap<>(); cache = new ConcurrentHashMap<>();
epochEntries = new TreeSet<>(); epochEntries = new ConcurrentSkipListSet<>();
// Created a singleThreadExecutor, so one cleanup will be running at a // Created a singleThreadExecutor, so one cleanup will be running at a
// time. // time.
ThreadFactory build = new ThreadFactoryBuilder().setDaemon(true) ThreadFactory build = new ThreadFactoryBuilder().setDaemon(true)