pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / src / java / org / apache / lucene / util / DoubleBarrelLRUCache.java
1 package org.apache.lucene.util;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * 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
20 import java.util.concurrent.ConcurrentHashMap;
21 import java.util.concurrent.atomic.AtomicInteger;
22 import java.util.Map;
23
24 /**
25  * Simple concurrent LRU cache, using a "double barrel"
26  * approach where two ConcurrentHashMaps record entries.
27  *
28  * <p>At any given time, one hash is primary and the other
29  * is secondary.  {@link #get} first checks primary, and if
30  * that's a miss, checks secondary.  If secondary has the
31  * entry, it's promoted to primary (<b>NOTE</b>: the key is
32  * cloned at this point).  Once primary is full, the
33  * secondary is cleared and the two are swapped.</p>
34  *
35  * <p>This is not as space efficient as other possible
36  * concurrent approaches (see LUCENE-2075): to achieve
37  * perfect LRU(N) it requires 2*N storage.  But, this
38  * approach is relatively simple and seems in practice to
39  * not grow unbounded in size when under hideously high
40  * load.</p>
41  *
42  * @lucene.internal
43  */
44
45 final public class DoubleBarrelLRUCache<K extends DoubleBarrelLRUCache.CloneableKey,V> {
46
47   public static abstract class CloneableKey {
48     @Override
49     abstract public Object clone();
50   }
51
52   private final Map<K,V> cache1;
53   private final Map<K,V> cache2;
54   private final AtomicInteger countdown;
55   private volatile boolean swapped;
56   private final int maxSize;
57
58   public DoubleBarrelLRUCache(int maxSize) {
59     this.maxSize = maxSize;
60     countdown = new AtomicInteger(maxSize);
61     cache1 = new ConcurrentHashMap<K,V>();
62     cache2 = new ConcurrentHashMap<K,V>();
63   }
64
65   @SuppressWarnings("unchecked") 
66   public V get(K key) {
67     final Map<K,V> primary;
68     final Map<K,V> secondary;
69     if (swapped) {
70       primary = cache2;
71       secondary = cache1;
72     } else {
73       primary = cache1;
74       secondary = cache2;
75     }
76
77     // Try primary first
78     V result = primary.get(key);
79     if (result == null) {
80       // Not found -- try secondary
81       result = secondary.get(key);
82       if (result != null) {
83         // Promote to primary
84         put((K) key.clone(), result);
85       }
86     }
87     return result;
88   }
89
90   public void put(K key, V value) {
91     final Map<K,V> primary;
92     final Map<K,V> secondary;
93     if (swapped) {
94       primary = cache2;
95       secondary = cache1;
96     } else {
97       primary = cache1;
98       secondary = cache2;
99     }
100     primary.put(key, value);
101
102     if (countdown.decrementAndGet() == 0) {
103       // Time to swap
104
105       // NOTE: there is saturation risk here, that the
106       // thread that's doing the clear() takes too long to
107       // do so, while other threads continue to add to
108       // primary, but in practice this seems not to be an
109       // issue (see LUCENE-2075 for benchmark & details)
110
111       // First, clear secondary
112       secondary.clear();
113
114       // Second, swap
115       swapped = !swapped;
116
117       // Third, reset countdown
118       countdown.set(maxSize);
119     }
120   }
121 }