View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import org.apache.commons.logging.Log;
22  import org.apache.commons.logging.LogFactory;
23  import org.apache.hadoop.hbase.classification.InterfaceAudience;
24  import org.apache.hadoop.hbase.io.HeapSize;
25  import org.apache.hadoop.hbase.util.Bytes;
26  import org.apache.hadoop.hbase.util.ClassSize;
27  
28  import java.util.ArrayList;
29  import java.util.Collection;
30  import java.util.HashSet;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Set;
34  
35  /**
36   * The LruHashMap is a memory-aware HashMap with a configurable maximum
37   * memory footprint.
38   * <p>
39   * It maintains an ordered list of all entries in the/ map ordered by
40   * access time.  When space needs to be freed becase the maximum has been
41   * reached, or the application has asked to free memory, entries will be
42   * evicted according to an LRU (least-recently-used) algorithm.  That is,
43   * those entries which have not been accessed the longest will be evicted
44   * first.
45   * <p>
46   * Both the Key and Value Objects used for this class must extend
47   * <code>HeapSize</code> in order to track heap usage.
48   * <p>
49   * This class contains internal synchronization and is thread-safe.
50   */
51  @InterfaceAudience.Private
52  public class LruHashMap<K extends HeapSize, V extends HeapSize>
53  implements HeapSize, Map<K,V> {
54  
55    private static final Log LOG = LogFactory.getLog(LruHashMap.class);
56  
57    /** The default size (in bytes) of the LRU */
58    private static final long DEFAULT_MAX_MEM_USAGE = 50000;
59    /** The default capacity of the hash table */
60    private static final int DEFAULT_INITIAL_CAPACITY = 16;
61    /** The maxmum capacity of the hash table */
62    private static final int MAXIMUM_CAPACITY = 1 << 30;
63    /** The default load factor to use */
64    private static final float DEFAULT_LOAD_FACTOR = 0.75f;
65  
66    /** Memory overhead of this Object (for HeapSize) */
67    private static final int OVERHEAD = 5 * Bytes.SIZEOF_LONG +
68      2 * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_FLOAT + 3 * ClassSize.REFERENCE +
69      1 * ClassSize.ARRAY;
70  
71    /** Load factor allowed (usually 75%) */
72    private final float loadFactor;
73    /** Number of key/vals in the map */
74    private int size;
75    /** Size at which we grow hash */
76    private int threshold;
77    /** Entries in the map */
78    private Entry [] entries;
79  
80    /** Pointer to least recently used entry */
81    private Entry<K,V> headPtr;
82    /** Pointer to most recently used entry */
83    private Entry<K,V> tailPtr;
84  
85    /** Maximum memory usage of this map */
86    private long memTotal = 0;
87    /** Amount of available memory */
88    private long memFree = 0;
89  
90    /** Number of successful (found) get() calls */
91    private long hitCount = 0;
92    /** Number of unsuccessful (not found) get() calls */
93    private long missCount = 0;
94  
95    /**
96     * Constructs a new, empty map with the specified initial capacity,
97     * load factor, and maximum memory usage.
98     *
99     * @param initialCapacity the initial capacity
100    * @param loadFactor the load factor
101    * @param maxMemUsage the maximum total memory usage
102    * @throws IllegalArgumentException if the initial capacity is less than one
103    * @throws IllegalArgumentException if the initial capacity is greater than
104    * the maximum capacity
105    * @throws IllegalArgumentException if the load factor is &lt;= 0
106    * @throws IllegalArgumentException if the max memory usage is too small
107    * to support the base overhead
108    */
109   public LruHashMap(int initialCapacity, float loadFactor,
110   long maxMemUsage) {
111     if (initialCapacity < 1) {
112       throw new IllegalArgumentException("Initial capacity must be > 0");
113     }
114     if (initialCapacity > MAXIMUM_CAPACITY) {
115       throw new IllegalArgumentException("Initial capacity is too large");
116     }
117     if (loadFactor <= 0 || Float.isNaN(loadFactor)) {
118       throw new IllegalArgumentException("Load factor must be > 0");
119     }
120     if (maxMemUsage <= (OVERHEAD + initialCapacity * ClassSize.REFERENCE)) {
121       throw new IllegalArgumentException("Max memory usage too small to " +
122       "support base overhead");
123     }
124 
125     /** Find a power of 2 >= initialCapacity */
126     int capacity = calculateCapacity(initialCapacity);
127     this.loadFactor = loadFactor;
128     this.threshold = calculateThreshold(capacity,loadFactor);
129     this.entries = new Entry[capacity];
130     this.memFree = maxMemUsage;
131     this.memTotal = maxMemUsage;
132     init();
133   }
134 
135   /**
136    * Constructs a new, empty map with the specified initial capacity and
137    * load factor, and default maximum memory usage.
138    *
139    * @param initialCapacity the initial capacity
140    * @param loadFactor the load factor
141    * @throws IllegalArgumentException if the initial capacity is less than one
142    * @throws IllegalArgumentException if the initial capacity is greater than
143    * the maximum capacity
144    * @throws IllegalArgumentException if the load factor is &lt;= 0
145    */
146   public LruHashMap(int initialCapacity, float loadFactor) {
147     this(initialCapacity, loadFactor, DEFAULT_MAX_MEM_USAGE);
148   }
149 
150   /**
151    * Constructs a new, empty map with the specified initial capacity and
152    * with the default load factor and maximum memory usage.
153    *
154    * @param initialCapacity the initial capacity
155    * @throws IllegalArgumentException if the initial capacity is less than one
156    * @throws IllegalArgumentException if the initial capacity is greater than
157    * the maximum capacity
158    */
159   public LruHashMap(int initialCapacity) {
160     this(initialCapacity, DEFAULT_LOAD_FACTOR, DEFAULT_MAX_MEM_USAGE);
161   }
162 
163   /**
164    * Constructs a new, empty map with the specified maximum memory usage
165    * and with default initial capacity and load factor.
166    *
167    * @param maxMemUsage the maximum total memory usage
168    * @throws IllegalArgumentException if the max memory usage is too small
169    * to support the base overhead
170    */
171   public LruHashMap(long maxMemUsage) {
172     this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR,
173     maxMemUsage);
174   }
175 
176   /**
177    * Constructs a new, empty map with the default initial capacity,
178    * load factor and maximum memory usage.
179    */
180   public LruHashMap() {
181     this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR,
182     DEFAULT_MAX_MEM_USAGE);
183   }
184 
185   //--------------------------------------------------------------------------
186   /**
187    * Get the currently available memory for this LRU in bytes.
188    * This is (maxAllowed - currentlyUsed).
189    *
190    * @return currently available bytes
191    */
192   public synchronized long getMemFree() {
193     return memFree;
194   }
195 
196   /**
197    * Get the maximum memory allowed for this LRU in bytes.
198    *
199    * @return maximum allowed bytes
200    */
201   public long getMemMax() {
202     return memTotal;
203   }
204 
205   /**
206    * Get the currently used memory for this LRU in bytes.
207    *
208    * @return currently used memory in bytes
209    */
210   public long getMemUsed() {
211     return (memTotal - getMemFree()); // FindBugs IS2_INCONSISTENT_SYNC
212   }
213 
214   /**
215    * Get the number of hits to the map.  This is the number of times
216    * a call to get() returns a matched key.
217    *
218    * @return number of hits
219    */
220   public long getHitCount() {
221     return hitCount;
222   }
223 
224   /**
225    * Get the number of misses to the map.  This is the number of times
226    * a call to get() returns null.
227    *
228    * @return number of misses
229    */
230   public synchronized long getMissCount() {
231     return missCount; // FindBugs IS2_INCONSISTENT_SYNC
232   }
233 
234   /**
235    * Get the hit ratio.  This is the number of hits divided by the
236    * total number of requests.
237    *
238    * @return hit ratio (double between 0 and 1)
239    */
240   public double getHitRatio() {
241     return (double)((double)hitCount/
242       ((double)(hitCount + getMissCount())));
243   }
244 
245   /**
246    * Free the requested amount of memory from the LRU map.
247    *
248    * This will do LRU eviction from the map until at least as much
249    * memory as requested is freed.  This does not affect the maximum
250    * memory usage parameter.
251    *
252    * @param requestedAmount memory to free from LRU in bytes
253    * @return actual amount of memory freed in bytes
254    */
255   public synchronized long freeMemory(long requestedAmount) throws Exception {
256     if(requestedAmount > (getMemUsed() - getMinimumUsage())) {
257       return clearAll();
258     }
259     long freedMemory = 0;
260     while(freedMemory < requestedAmount) {
261       freedMemory += evictFromLru();
262     }
263     return freedMemory;
264   }
265 
266   /**
267    * The total memory usage of this map
268    *
269    * @return memory usage of map in bytes
270    */
271   public long heapSize() {
272     return (memTotal - getMemFree());
273   }
274 
275   //--------------------------------------------------------------------------
276   /**
277    * Retrieves the value associated with the specified key.
278    *
279    * If an entry is found, it is updated in the LRU as the most recently
280    * used (last to be evicted) entry in the map.
281    *
282    * @param key the key
283    * @return the associated value, or null if none found
284    * @throws NullPointerException if key is null
285    */
286   public synchronized V get(Object key) {
287     checkKey((K)key);
288     int hash = hash(key);
289     int i = hashIndex(hash, entries.length);
290     Entry<K,V> e = entries[i];
291     while (true) {
292       if (e == null) {
293         missCount++;
294         return null;
295       }
296       if (e.hash == hash && isEqual(key, e.key))  {
297         // Hit!  Update position in LRU
298         hitCount++;
299         updateLru(e);
300         return e.value;
301       }
302       e = e.next;
303     }
304   }
305 
306   /**
307    * Insert a key-value mapping into the map.
308    *
309    * Entry will be inserted as the most recently used.
310    *
311    * Both the key and value are required to be Objects and must
312    * implement the HeapSize interface.
313    *
314    * @param key the key
315    * @param value the value
316    * @return the value that was previously mapped to this key, null if none
317    * @throws UnsupportedOperationException if either objects do not
318    * implement HeapSize
319    * @throws NullPointerException if the key or value is null
320    */
321   public synchronized V put(K key, V value) {
322     checkKey(key);
323     checkValue(value);
324     int hash = hash(key);
325     int i = hashIndex(hash, entries.length);
326 
327     // For old values
328     for (Entry<K,V> e = entries[i]; e != null; e = e.next) {
329       if (e.hash == hash && isEqual(key, e.key)) {
330         V oldValue = e.value;
331         long memChange = e.replaceValue(value);
332         checkAndFreeMemory(memChange);
333         // If replacing an old value for this key, update in LRU
334         updateLru(e);
335         return oldValue;
336       }
337     }
338     long memChange = addEntry(hash, key, value, i);
339     checkAndFreeMemory(memChange);
340     return null;
341   }
342 
343   /**
344    * Deletes the mapping for the specified key if it exists.
345    *
346    * @param key the key of the entry to be removed from the map
347    * @return the value associated with the specified key, or null
348    * if no mapping exists.
349    */
350   public synchronized V remove(Object key) {
351     Entry<K,V> e = removeEntryForKey((K)key);
352     if(e == null) return null;
353     // Add freed memory back to available
354     memFree += e.heapSize();
355     return e.value;
356   }
357 
358   /**
359    * Gets the size (number of entries) of the map.
360    *
361    * @return size of the map
362    */
363   public int size() {
364     return size;
365   }
366 
367   /**
368    * Checks whether the map is currently empty.
369    *
370    * @return true if size of map is zero
371    */
372   public boolean isEmpty() {
373     return size == 0;
374   }
375 
376   /**
377    * Clears all entries from the map.
378    *
379    * This frees all entries, tracking memory usage along the way.
380    * All references to entries are removed so they can be GC'd.
381    */
382   public synchronized void clear() {
383     memFree += clearAll();
384   }
385 
386   //--------------------------------------------------------------------------
387   /**
388    * Checks whether there is a value in the map for the specified key.
389    *
390    * Does not affect the LRU.
391    *
392    * @param key the key to check
393    * @return true if the map contains a value for this key, false if not
394    * @throws NullPointerException if the key is null
395    */
396   public synchronized boolean containsKey(Object key) {
397     checkKey((K)key);
398     int hash = hash(key);
399     int i = hashIndex(hash, entries.length);
400     Entry e = entries[i];
401     while (e != null) {
402       if (e.hash == hash && isEqual(key, e.key))
403           return true;
404       e = e.next;
405     }
406     return false;
407   }
408 
409   /**
410    * Checks whether this is a mapping which contains the specified value.
411    *
412    * Does not affect the LRU.  This is an inefficient operation.
413    *
414    * @param value the value to check
415    * @return true if the map contains an entry for this value, false
416    * if not
417    * @throws NullPointerException if the value is null
418    */
419   public synchronized boolean containsValue(Object value) {
420     checkValue((V)value);
421     Entry[] tab = entries;
422     for (int i = 0; i < tab.length ; i++)
423       for (Entry e = tab[i] ; e != null ; e = e.next)
424           if (value.equals(e.value))
425             return true;
426     return false;
427   }
428 
429   //--------------------------------------------------------------------------
430   /**
431    * Enforces key constraints.  Null keys are not permitted and key must
432    * implement HeapSize.  It should not be necessary to verify the second
433    * constraint because that's enforced on instantiation?
434    *
435    * Can add other constraints in the future.
436    *
437    * @param key the key
438    * @throws NullPointerException if the key is null
439    * @throws UnsupportedOperationException if the key class does not
440    * implement the HeapSize interface
441    */
442   private void checkKey(K key) {
443     if(key == null) {
444       throw new NullPointerException("null keys are not allowed");
445     }
446   }
447 
448   /**
449    * Enforces value constraints.  Null values are not permitted and value must
450    * implement HeapSize.  It should not be necessary to verify the second
451    * constraint because that's enforced on instantiation?
452    *
453    * Can add other contraints in the future.
454    *
455    * @param value the value
456    * @throws NullPointerException if the value is null
457    * @throws UnsupportedOperationException if the value class does not
458    * implement the HeapSize interface
459    */
460   private void checkValue(V value) {
461     if(value == null) {
462       throw new NullPointerException("null values are not allowed");
463     }
464   }
465 
466   /**
467    * Returns the minimum memory usage of the base map structure.
468    *
469    * @return baseline memory overhead of object in bytes
470    */
471   private long getMinimumUsage() {
472     return OVERHEAD + (entries.length * ClassSize.REFERENCE);
473   }
474 
475   //--------------------------------------------------------------------------
476   /**
477    * Evicts and frees based on LRU until at least as much memory as requested
478    * is available.
479    *
480    * @param memNeeded the amount of memory needed in bytes
481    */
482   private void checkAndFreeMemory(long memNeeded) {
483     while(memFree < memNeeded) {
484       evictFromLru();
485     }
486     memFree -= memNeeded;
487   }
488 
489   /**
490    * Evicts based on LRU.  This removes all references and updates available
491    * memory.
492    *
493    * @return amount of memory freed in bytes
494    */
495   private long evictFromLru() {
496     long freed = headPtr.heapSize();
497     memFree += freed;
498     removeEntry(headPtr);
499     return freed;
500   }
501 
502   /**
503    * Moves the specified entry to the most recently used slot of the
504    * LRU.  This is called whenever an entry is fetched.
505    *
506    * @param e entry that was accessed
507    */
508   private void updateLru(Entry<K,V> e) {
509     Entry<K,V> prev = e.getPrevPtr();
510     Entry<K,V> next = e.getNextPtr();
511     if(next != null) {
512       if(prev != null) {
513         prev.setNextPtr(next);
514         next.setPrevPtr(prev);
515       } else {
516         headPtr = next;
517         headPtr.setPrevPtr(null);
518       }
519       e.setNextPtr(null);
520       e.setPrevPtr(tailPtr);
521       tailPtr.setNextPtr(e);
522       tailPtr = e;
523     }
524   }
525 
526   /**
527    * Removes the specified entry from the map and LRU structure.
528    *
529    * @param entry entry to be removed
530    */
531   private void removeEntry(Entry<K,V> entry) {
532     K k = entry.key;
533     int hash = entry.hash;
534     int i = hashIndex(hash, entries.length);
535     Entry<K,V> prev = entries[i];
536     Entry<K,V> e = prev;
537 
538     while (e != null) {
539       Entry<K,V> next = e.next;
540       if (e.hash == hash && isEqual(k, e.key)) {
541           size--;
542           if (prev == e) {
543             entries[i] = next;
544           } else {
545             prev.next = next;
546           }
547 
548           Entry<K,V> prevPtr = e.getPrevPtr();
549           Entry<K,V> nextPtr = e.getNextPtr();
550 
551           if(prevPtr != null && nextPtr != null) {
552             prevPtr.setNextPtr(nextPtr);
553             nextPtr.setPrevPtr(prevPtr);
554           } else if(prevPtr != null) {
555             tailPtr = prevPtr;
556             prevPtr.setNextPtr(null);
557           } else if(nextPtr != null) {
558             headPtr = nextPtr;
559             nextPtr.setPrevPtr(null);
560           }
561 
562           return;
563       }
564       prev = e;
565       e = next;
566     }
567   }
568 
569   /**
570    * Removes and returns the entry associated with the specified
571    * key.
572    *
573    * @param key key of the entry to be deleted
574    * @return entry that was removed, or null if none found
575    */
576   private Entry<K,V> removeEntryForKey(K key) {
577     int hash = hash(key);
578     int i = hashIndex(hash, entries.length);
579     Entry<K,V> prev = entries[i];
580     Entry<K,V> e = prev;
581 
582     while (e != null) {
583       Entry<K,V> next = e.next;
584       if (e.hash == hash && isEqual(key, e.key)) {
585           size--;
586           if (prev == e) {
587             entries[i] = next;
588           } else {
589             prev.next = next;
590           }
591 
592           // Updating LRU
593           Entry<K,V> prevPtr = e.getPrevPtr();
594           Entry<K,V> nextPtr = e.getNextPtr();
595           if(prevPtr != null && nextPtr != null) {
596             prevPtr.setNextPtr(nextPtr);
597             nextPtr.setPrevPtr(prevPtr);
598           } else if(prevPtr != null) {
599             tailPtr = prevPtr;
600             prevPtr.setNextPtr(null);
601           } else if(nextPtr != null) {
602             headPtr = nextPtr;
603             nextPtr.setPrevPtr(null);
604           }
605 
606           return e;
607       }
608       prev = e;
609       e = next;
610     }
611 
612     return e;
613   }
614 
615  /**
616   * Adds a new entry with the specified key, value, hash code, and
617   * bucket index to the map.
618   *
619   * Also puts it in the bottom (most-recent) slot of the list and
620   * checks to see if we need to grow the array.
621   *
622   * @param hash hash value of key
623   * @param key the key
624   * @param value the value
625   * @param bucketIndex index into hash array to store this entry
626   * @return the amount of heap size used to store the new entry
627   */
628   private long addEntry(int hash, K key, V value, int bucketIndex) {
629     Entry<K,V> e = entries[bucketIndex];
630     Entry<K,V> newE = new Entry<K,V>(hash, key, value, e, tailPtr);
631     entries[bucketIndex] = newE;
632     // add as most recently used in lru
633     if (size == 0) {
634       headPtr = newE;
635       tailPtr = newE;
636     } else {
637       newE.setPrevPtr(tailPtr);
638       tailPtr.setNextPtr(newE);
639       tailPtr = newE;
640     }
641     // Grow table if we are past the threshold now
642     if (size++ >= threshold) {
643       growTable(2 * entries.length);
644     }
645     return newE.heapSize();
646   }
647 
648   /**
649    * Clears all the entries in the map.  Tracks the amount of memory being
650    * freed along the way and returns the total.
651    *
652    * Cleans up all references to allow old entries to be GC'd.
653    *
654    * @return total memory freed in bytes
655    */
656   private long clearAll() {
657     Entry cur;
658     long freedMemory = 0;
659     for(int i=0; i<entries.length; i++) {
660       cur = entries[i];
661       while(cur != null) {
662         freedMemory += cur.heapSize();
663         cur = cur.next;
664       }
665       entries[i] = null;
666     }
667     headPtr = null;
668     tailPtr = null;
669     size = 0;
670     return freedMemory;
671   }
672 
673   //--------------------------------------------------------------------------
674   /**
675    * Recreates the entire contents of the hashmap into a new array
676    * with double the capacity.  This method is called when the number of
677    * keys in the map reaches the current threshold.
678    *
679    * @param newCapacity the new size of the hash entries
680    */
681   private void growTable(int newCapacity) {
682     Entry [] oldTable = entries;
683     int oldCapacity = oldTable.length;
684 
685     // Do not allow growing the table beyond the max capacity
686     if (oldCapacity == MAXIMUM_CAPACITY) {
687       threshold = Integer.MAX_VALUE;
688       return;
689     }
690 
691     // Determine how much additional space will be required to grow the array
692     long requiredSpace = (newCapacity - oldCapacity) * ClassSize.REFERENCE;
693 
694     // Verify/enforce we have sufficient memory to grow
695     checkAndFreeMemory(requiredSpace);
696 
697     Entry [] newTable = new Entry[newCapacity];
698 
699     // Transfer existing entries to new hash table
700     for(int i=0; i < oldCapacity; i++) {
701       Entry<K,V> entry = oldTable[i];
702       if(entry != null) {
703         // Set to null for GC
704         oldTable[i] = null;
705         do {
706           Entry<K,V> next = entry.next;
707           int idx = hashIndex(entry.hash, newCapacity);
708           entry.next = newTable[idx];
709           newTable[idx] = entry;
710           entry = next;
711         } while(entry != null);
712       }
713     }
714 
715     entries = newTable;
716     threshold = (int)(newCapacity * loadFactor);
717   }
718 
719   /**
720    * Gets the hash code for the specified key.
721    * This implementation uses the additional hashing routine
722    * from JDK 1.4.
723    *
724    * @param key the key to get a hash value for
725    * @return the hash value
726    */
727   private int hash(Object key) {
728     int h = key.hashCode();
729     h += ~(h << 9);
730     h ^=  (h >>> 14);
731     h +=  (h << 4);
732     h ^=  (h >>> 10);
733     return h;
734   }
735 
736   /**
737    * Compares two objects for equality.  Method uses equals method and
738    * assumes neither value is null.
739    *
740    * @param x the first value
741    * @param y the second value
742    * @return true if equal
743    */
744   private boolean isEqual(Object x, Object y) {
745     return (x == y || x.equals(y));
746   }
747 
748   /**
749    * Determines the index into the current hash table for the specified
750    * hashValue.
751    *
752    * @param hashValue the hash value
753    * @param length the current number of hash buckets
754    * @return the index of the current hash array to use
755    */
756   private int hashIndex(int hashValue, int length) {
757     return hashValue & (length - 1);
758   }
759 
760   /**
761    * Calculates the capacity of the array backing the hash
762    * by normalizing capacity to a power of 2 and enforcing
763    * capacity limits.
764    *
765    * @param proposedCapacity the proposed capacity
766    * @return the normalized capacity
767    */
768   private int calculateCapacity(int proposedCapacity) {
769     int newCapacity = 1;
770     if(proposedCapacity > MAXIMUM_CAPACITY) {
771       newCapacity = MAXIMUM_CAPACITY;
772     } else {
773       while(newCapacity < proposedCapacity) {
774         newCapacity <<= 1;
775       }
776       if(newCapacity > MAXIMUM_CAPACITY) {
777         newCapacity = MAXIMUM_CAPACITY;
778       }
779     }
780     return newCapacity;
781   }
782 
783   /**
784    * Calculates the threshold of the map given the capacity and load
785    * factor.  Once the number of entries in the map grows to the
786    * threshold we will double the size of the array.
787    *
788    * @param capacity the size of the array
789    * @param factor the load factor of the hash
790    */
791   private int calculateThreshold(int capacity, float factor) {
792     return (int)(capacity * factor);
793   }
794 
795   /**
796    * Set the initial heap usage of this class.  Includes class variable
797    * overhead and the entry array.
798    */
799   private void init() {
800     memFree -= OVERHEAD;
801     memFree -= (entries.length * ClassSize.REFERENCE);
802   }
803 
804   //--------------------------------------------------------------------------
805   /**
806    * Debugging function that returns a List sorted by access time.
807    *
808    * The order is oldest to newest (first in list is next to be evicted).
809    *
810    * @return Sorted list of entries
811    */
812   public List<Entry<K,V>> entryLruList() {
813     List<Entry<K,V>> entryList = new ArrayList<Entry<K,V>>();
814     Entry<K,V> entry = headPtr;
815     while(entry != null) {
816       entryList.add(entry);
817       entry = entry.getNextPtr();
818     }
819     return entryList;
820   }
821 
822   /**
823    * Debugging function that returns a Set of all entries in the hash table.
824    *
825    * @return Set of entries in hash
826    */
827   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
828       justification="Unused debugging function that reads only")
829   public Set<Entry<K,V>> entryTableSet() {
830     Set<Entry<K,V>> entrySet = new HashSet<Entry<K,V>>();
831     Entry [] table = entries; // FindBugs IS2_INCONSISTENT_SYNC
832     for(int i=0;i<table.length;i++) {
833       for(Entry e = table[i]; e != null; e = e.next) {
834         entrySet.add(e);
835       }
836     }
837     return entrySet;
838   }
839 
840   /**
841    * Get the head of the linked list (least recently used).
842    *
843    * @return head of linked list
844    */
845   public Entry getHeadPtr() {
846     return headPtr;
847   }
848 
849   /**
850    * Get the tail of the linked list (most recently used).
851    *
852    * @return tail of linked list
853    */
854   public Entry getTailPtr() {
855     return tailPtr;
856   }
857 
858   //--------------------------------------------------------------------------
859   /**
860    * To best optimize this class, some of the methods that are part of a
861    * Map implementation are not supported.  This is primarily related
862    * to being able to get Sets and Iterators of this map which require
863    * significant overhead and code complexity to support and are
864    * unnecessary for the requirements of this class.
865    */
866 
867   /**
868    * Intentionally unimplemented.
869    */
870   public Set<Map.Entry<K,V>> entrySet() {
871     throw new UnsupportedOperationException(
872     "entrySet() is intentionally unimplemented");
873   }
874 
875   /**
876    * Intentionally unimplemented.
877    */
878   public boolean equals(Object o) {
879     throw new UnsupportedOperationException(
880     "equals(Object) is intentionally unimplemented");
881   }
882 
883   /**
884    * Intentionally unimplemented.
885    */
886   public int hashCode() {
887     throw new UnsupportedOperationException(
888     "hashCode(Object) is intentionally unimplemented");
889   }
890 
891   /**
892    * Intentionally unimplemented.
893    */
894   public Set<K> keySet() {
895     throw new UnsupportedOperationException(
896     "keySet() is intentionally unimplemented");
897   }
898 
899   /**
900    * Intentionally unimplemented.
901    */
902   public void putAll(Map<? extends K, ? extends V> m) {
903     throw new UnsupportedOperationException(
904     "putAll() is intentionally unimplemented");
905   }
906 
907   /**
908    * Intentionally unimplemented.
909    */
910   public Collection<V> values() {
911     throw new UnsupportedOperationException(
912     "values() is intentionally unimplemented");
913   }
914 
915   //--------------------------------------------------------------------------
916   /**
917    * Entry to store key/value mappings.
918    * <p>
919    * Contains previous and next pointers for the doubly linked-list which is
920    * used for LRU eviction.
921    * <p>
922    * Instantiations of this class are memory aware.  Both the key and value
923    * classes used must also implement <code>HeapSize</code>.
924    */
925   protected static class Entry<K extends HeapSize, V extends HeapSize>
926   implements Map.Entry<K,V>, HeapSize {
927     /** The baseline overhead memory usage of this class */
928     static final int OVERHEAD = 1 * Bytes.SIZEOF_LONG +
929       5 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT;
930 
931     /** The key */
932     protected final K key;
933     /** The value */
934     protected V value;
935     /** The hash value for this entries key */
936     protected final int hash;
937     /** The next entry in the hash chain (for collisions) */
938     protected Entry<K,V> next;
939 
940     /** The previous entry in the LRU list (towards LRU) */
941     protected Entry<K,V> prevPtr;
942     /** The next entry in the LRU list (towards MRU) */
943     protected Entry<K,V> nextPtr;
944 
945     /** The precomputed heap size of this entry */
946     protected long heapSize;
947 
948     /**
949      * Create a new entry.
950      *
951      * @param h the hash value of the key
952      * @param k the key
953      * @param v the value
954      * @param nextChainPtr the next entry in the hash chain, null if none
955      * @param prevLruPtr the previous entry in the LRU
956      */
957     Entry(int h, K k, V v, Entry<K,V> nextChainPtr, Entry<K,V> prevLruPtr) {
958       value = v;
959       next = nextChainPtr;
960       key = k;
961       hash = h;
962       prevPtr = prevLruPtr;
963       nextPtr = null;
964       // Pre-compute heap size
965       heapSize = OVERHEAD + k.heapSize() + v.heapSize();
966     }
967 
968     /**
969      * Get the key of this entry.
970      *
971      * @return the key associated with this entry
972      */
973     public K getKey() {
974       return key;
975     }
976 
977     /**
978      * Get the value of this entry.
979      *
980      * @return the value currently associated with this entry
981      */
982     public V getValue() {
983       return value;
984     }
985 
986     /**
987      * Set the value of this entry.
988      *
989      * It is not recommended to use this method when changing the value.
990      * Rather, using <code>replaceValue</code> will return the difference
991      * in heap usage between the previous and current values.
992      *
993      * @param newValue the new value to associate with this entry
994      * @return the value previously associated with this entry
995      */
996     public V setValue(V newValue) {
997       V oldValue = value;
998       value = newValue;
999       return oldValue;
1000     }
1001 
1002     /**
1003      * Replace the value of this entry.
1004      *
1005      * Computes and returns the difference in heap size when changing
1006      * the value associated with this entry.
1007      *
1008      * @param newValue the new value to associate with this entry
1009      * @return the change in heap usage of this entry in bytes
1010      */
1011     protected long replaceValue(V newValue) {
1012       long sizeDiff = newValue.heapSize() - value.heapSize();
1013       value = newValue;
1014       heapSize += sizeDiff;
1015       return sizeDiff;
1016     }
1017 
1018     /**
1019      * Returns true is the specified entry has the same key and the
1020      * same value as this entry.
1021      *
1022      * @param o entry to test against current
1023      * @return true is entries have equal key and value, false if no
1024      */
1025     public boolean equals(Object o) {
1026       if (!(o instanceof Map.Entry))
1027           return false;
1028       Map.Entry e = (Map.Entry)o;
1029       Object k1 = getKey();
1030       Object k2 = e.getKey();
1031       if (k1 == k2 || (k1 != null && k1.equals(k2))) {
1032           Object v1 = getValue();
1033           Object v2 = e.getValue();
1034           if (v1 == v2 || (v1 != null && v1.equals(v2)))
1035             return true;
1036       }
1037       return false;
1038     }
1039 
1040     /**
1041      * Returns the hash code of the entry by xor'ing the hash values
1042      * of the key and value of this entry.
1043      *
1044      * @return hash value of this entry
1045      */
1046     public int hashCode() {
1047       return (key.hashCode() ^ value.hashCode());
1048     }
1049 
1050     /**
1051      * Returns String representation of the entry in form "key=value"
1052      *
1053      * @return string value of entry
1054      */
1055     public String toString() {
1056       return getKey() + "=" + getValue();
1057     }
1058 
1059     //------------------------------------------------------------------------
1060     /**
1061      * Sets the previous pointer for the entry in the LRU.
1062      * @param prevPtr previous entry
1063      */
1064     protected void setPrevPtr(Entry<K,V> prevPtr){
1065       this.prevPtr = prevPtr;
1066     }
1067 
1068     /**
1069      * Returns the previous pointer for the entry in the LRU.
1070      * @return previous entry
1071      */
1072     protected Entry<K,V> getPrevPtr(){
1073       return prevPtr;
1074     }
1075 
1076     /**
1077      * Sets the next pointer for the entry in the LRU.
1078      * @param nextPtr next entry
1079      */
1080     protected void setNextPtr(Entry<K,V> nextPtr){
1081       this.nextPtr = nextPtr;
1082     }
1083 
1084     /**
1085      * Returns the next pointer for the entry in teh LRU.
1086      * @return next entry
1087      */
1088     protected Entry<K,V> getNextPtr(){
1089       return nextPtr;
1090     }
1091 
1092     /**
1093      * Returns the pre-computed and "deep" size of the Entry
1094      * @return size of the entry in bytes
1095      */
1096     public long heapSize() {
1097       return heapSize;
1098     }
1099   }
1100 }
1101 
1102