|
@@ -26,6 +26,7 @@ import java.util.Collections;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.NavigableSet;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.SortedMap;
|
|
import java.util.SortedMap;
|
|
import java.util.TreeMap;
|
|
import java.util.TreeMap;
|
|
@@ -36,6 +37,7 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|
import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.ThreadFactory;
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -131,19 +133,73 @@ public class HistoryFileManager extends AbstractService {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- static class JobListCache {
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Wrapper around {@link ConcurrentSkipListMap} that maintains size along
|
|
|
|
+ * side for O(1) size() implementation for use in JobListCache.
|
|
|
|
+ *
|
|
|
|
+ * Note: The size is not updated atomically with changes additions/removals.
|
|
|
|
+ * This race can lead to size() returning an incorrect size at times.
|
|
|
|
+ */
|
|
|
|
+ static class JobIdHistoryFileInfoMap {
|
|
private ConcurrentSkipListMap<JobId, HistoryFileInfo> cache;
|
|
private ConcurrentSkipListMap<JobId, HistoryFileInfo> cache;
|
|
|
|
+ private AtomicInteger mapSize;
|
|
|
|
+
|
|
|
|
+ JobIdHistoryFileInfoMap() {
|
|
|
|
+ cache = new ConcurrentSkipListMap<JobId, HistoryFileInfo>();
|
|
|
|
+ mapSize = new AtomicInteger();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public HistoryFileInfo putIfAbsent(JobId key, HistoryFileInfo value) {
|
|
|
|
+ HistoryFileInfo ret = cache.putIfAbsent(key, value);
|
|
|
|
+ if (ret == null) {
|
|
|
|
+ mapSize.incrementAndGet();
|
|
|
|
+ }
|
|
|
|
+ return ret;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public HistoryFileInfo remove(JobId key) {
|
|
|
|
+ HistoryFileInfo ret = cache.remove(key);
|
|
|
|
+ if (ret != null) {
|
|
|
|
+ mapSize.decrementAndGet();
|
|
|
|
+ }
|
|
|
|
+ return ret;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Returns the recorded size of the internal map. Note that this could be out
|
|
|
|
+ * of sync with the actual size of the map
|
|
|
|
+ * @return "recorded" size
|
|
|
|
+ */
|
|
|
|
+ public int size() {
|
|
|
|
+ return mapSize.get();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public HistoryFileInfo get(JobId key) {
|
|
|
|
+ return cache.get(key);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public NavigableSet<JobId> navigableKeySet() {
|
|
|
|
+ return cache.navigableKeySet();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Collection<HistoryFileInfo> values() {
|
|
|
|
+ return cache.values();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static class JobListCache {
|
|
|
|
+ private JobIdHistoryFileInfoMap cache;
|
|
private int maxSize;
|
|
private int maxSize;
|
|
private long maxAge;
|
|
private long maxAge;
|
|
|
|
|
|
public JobListCache(int maxSize, long maxAge) {
|
|
public JobListCache(int maxSize, long maxAge) {
|
|
this.maxSize = maxSize;
|
|
this.maxSize = maxSize;
|
|
this.maxAge = maxAge;
|
|
this.maxAge = maxAge;
|
|
- this.cache = new ConcurrentSkipListMap<JobId, HistoryFileInfo>();
|
|
|
|
|
|
+ this.cache = new JobIdHistoryFileInfoMap();
|
|
}
|
|
}
|
|
|
|
|
|
public HistoryFileInfo addIfAbsent(HistoryFileInfo fileInfo) {
|
|
public HistoryFileInfo addIfAbsent(HistoryFileInfo fileInfo) {
|
|
- JobId jobId = fileInfo.getJobIndexInfo().getJobId();
|
|
|
|
|
|
+ JobId jobId = fileInfo.getJobId();
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Adding " + jobId + " to job list cache with "
|
|
LOG.debug("Adding " + jobId + " to job list cache with "
|
|
+ fileInfo.getJobIndexInfo());
|
|
+ fileInfo.getJobIndexInfo());
|