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.util;
20  
21  import org.apache.hadoop.classification.InterfaceAudience;
22  import org.apache.hadoop.classification.InterfaceStability;
23  
24  import java.util.ArrayList;
25  import java.util.Collection;
26  import java.util.HashMap;
27  import java.util.HashSet;
28  import java.util.LinkedList;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Set;
32  import java.util.concurrent.ConcurrentHashMap;
33  import java.util.concurrent.ConcurrentLinkedQueue;
34  import java.util.concurrent.CopyOnWriteArrayList;
35  import java.util.concurrent.atomic.AtomicInteger;
36  
37  /**
38   *
39   * The <code>PoolMap</code> maps a key to a collection of values, the elements
40   * of which are managed by a pool. In effect, that collection acts as a shared
41   * pool of resources, access to which is closely controlled as per the semantics
42   * of the pool.
43   *
44   * <p>
45   * In case the size of the pool is set to a non-zero positive number, that is
46   * used to cap the number of resources that a pool may contain for any given
47   * key. A size of {@link Integer#MAX_VALUE} is interpreted as an unbounded pool.
48   * </p>
49   *
50   * @param <K>
51   *          the type of the key to the resource
52   * @param <V>
53   *          the type of the resource being pooled
54   */
55  @InterfaceAudience.Public
56  @InterfaceStability.Evolving
57  public class PoolMap<K, V> implements Map<K, V> {
58    private PoolType poolType;
59  
60    private int poolMaxSize;
61  
62    private Map<K, Pool<V>> pools = new ConcurrentHashMap<K, Pool<V>>();
63  
64    public PoolMap(PoolType poolType) {
65      this.poolType = poolType;
66    }
67  
68    public PoolMap(PoolType poolType, int poolMaxSize) {
69      this.poolType = poolType;
70      this.poolMaxSize = poolMaxSize;
71    }
72  
73    @Override
74    public V get(Object key) {
75      Pool<V> pool = pools.get(key);
76      return pool != null ? pool.get() : null;
77    }
78  
79    @Override
80    public V put(K key, V value) {
81      Pool<V> pool = pools.get(key);
82      if (pool == null) {
83        pools.put(key, pool = createPool());
84      }
85      return pool != null ? pool.put(value) : null;
86    }
87  
88    @SuppressWarnings("unchecked")
89    @Override
90    public V remove(Object key) {
91      Pool<V> pool = pools.remove(key);
92      if (pool != null) {
93        remove((K) key, pool.get());
94      }
95      return null;
96    }
97  
98    public boolean remove(K key, V value) {
99      Pool<V> pool = pools.get(key);
100     boolean res = false;
101     if (pool != null) {
102       res = pool.remove(value);
103       if (res && pool.size() == 0) {
104         pools.remove(key);
105       }
106     }
107     return res;
108   }
109 
110   @Override
111   public Collection<V> values() {
112     Collection<V> values = new ArrayList<V>();
113     for (Pool<V> pool : pools.values()) {
114       Collection<V> poolValues = pool.values();
115       if (poolValues != null) {
116         values.addAll(poolValues);
117       }
118     }
119     return values;
120   }
121 
122   public Collection<V> values(K key) {
123     Collection<V> values = new ArrayList<V>();
124     Pool<V> pool = pools.get(key);
125     if (pool != null) {
126       Collection<V> poolValues = pool.values();
127       if (poolValues != null) {
128         values.addAll(poolValues);
129       }
130     }
131     return values;
132   }
133 
134 
135   @Override
136   public boolean isEmpty() {
137     return pools.isEmpty();
138   }
139 
140   @Override
141   public int size() {
142     return pools.size();
143   }
144 
145   public int size(K key) {
146     Pool<V> pool = pools.get(key);
147     return pool != null ? pool.size() : 0;
148   }
149 
150   @Override
151   public boolean containsKey(Object key) {
152     return pools.containsKey(key);
153   }
154 
155   @Override
156   public boolean containsValue(Object value) {
157     if (value == null) {
158       return false;
159     }
160     for (Pool<V> pool : pools.values()) {
161       if (value.equals(pool.get())) {
162         return true;
163       }
164     }
165     return false;
166   }
167 
168   @Override
169   public void putAll(Map<? extends K, ? extends V> map) {
170     for (Map.Entry<? extends K, ? extends V> entry : map.entrySet()) {
171       put(entry.getKey(), entry.getValue());
172     }
173   }
174 
175   @Override
176   public void clear() {
177     for (Pool<V> pool : pools.values()) {
178       pool.clear();
179     }
180     pools.clear();
181   }
182 
183   @Override
184   public Set<K> keySet() {
185     return pools.keySet();
186   }
187 
188   @Override
189   public Set<Map.Entry<K, V>> entrySet() {
190     Set<Map.Entry<K, V>> entries = new HashSet<Entry<K, V>>();
191     for (Map.Entry<K, Pool<V>> poolEntry : pools.entrySet()) {
192       final K poolKey = poolEntry.getKey();
193       final Pool<V> pool = poolEntry.getValue();
194       if (pool != null) {
195         for (final V poolValue : pool.values()) {
196           entries.add(new Map.Entry<K, V>() {
197             @Override
198             public K getKey() {
199               return poolKey;
200             }
201 
202             @Override
203             public V getValue() {
204               return poolValue;
205             }
206 
207             @Override
208             public V setValue(V value) {
209               return pool.put(value);
210             }
211           });
212         }
213       }
214     }
215     return null;
216   }
217 
218   protected interface Pool<R> {
219     public R get();
220 
221     public R put(R resource);
222 
223     public boolean remove(R resource);
224 
225     public void clear();
226 
227     public Collection<R> values();
228 
229     public int size();
230   }
231 
232   public enum PoolType {
233     Reusable, ThreadLocal, RoundRobin;
234 
235     public static PoolType valueOf(String poolTypeName,
236         PoolType defaultPoolType, PoolType... allowedPoolTypes) {
237       PoolType poolType = PoolType.fuzzyMatch(poolTypeName);
238       if (poolType != null) {
239         boolean allowedType = false;
240         if (poolType.equals(defaultPoolType)) {
241           allowedType = true;
242         } else {
243           if (allowedPoolTypes != null) {
244             for (PoolType allowedPoolType : allowedPoolTypes) {
245               if (poolType.equals(allowedPoolType)) {
246                 allowedType = true;
247                 break;
248               }
249             }
250           }
251         }
252         if (!allowedType) {
253           poolType = null;
254         }
255       }
256       return (poolType != null) ? poolType : defaultPoolType;
257     }
258 
259     public static String fuzzyNormalize(String name) {
260       return name != null ? name.replaceAll("-", "").trim().toLowerCase() : "";
261     }
262 
263     public static PoolType fuzzyMatch(String name) {
264       for (PoolType poolType : values()) {
265         if (fuzzyNormalize(name).equals(fuzzyNormalize(poolType.name()))) {
266           return poolType;
267         }
268       }
269       return null;
270     }
271   }
272 
273   protected Pool<V> createPool() {
274     switch (poolType) {
275     case Reusable:
276       return new ReusablePool<V>(poolMaxSize);
277     case RoundRobin:
278       return new RoundRobinPool<V>(poolMaxSize);
279     case ThreadLocal:
280       return new ThreadLocalPool<V>();
281     }
282     return null;
283   }
284 
285   /**
286    * The <code>ReusablePool</code> represents a {@link PoolMap.Pool} that builds
287    * on the {@link LinkedList} class. It essentially allows resources to be
288    * checked out, at which point it is removed from this pool. When the resource
289    * is no longer required, it should be returned to the pool in order to be
290    * reused.
291    *
292    * <p>
293    * If {@link #maxSize} is set to {@link Integer#MAX_VALUE}, then the size of
294    * the pool is unbounded. Otherwise, it caps the number of consumers that can
295    * check out a resource from this pool to the (non-zero positive) value
296    * specified in {@link #maxSize}.
297    * </p>
298    *
299    * @param <R>
300    *          the type of the resource
301    */
302   @SuppressWarnings("serial")
303   public class ReusablePool<R> extends ConcurrentLinkedQueue<R> implements Pool<R> {
304     private int maxSize;
305 
306     public ReusablePool(int maxSize) {
307       this.maxSize = maxSize;
308 
309     }
310 
311     @Override
312     public R get() {
313       return poll();
314     }
315 
316     @Override
317     public R put(R resource) {
318       if (super.size() < maxSize) {
319         add(resource);
320       }
321       return null;
322     }
323 
324     @Override
325     public Collection<R> values() {
326       return this;
327     }
328   }
329 
330   /**
331    * The <code>RoundRobinPool</code> represents a {@link PoolMap.Pool}, which
332    * stores its resources in an {@link ArrayList}. It load-balances access to
333    * its resources by returning a different resource every time a given key is
334    * looked up.
335    *
336    * <p>
337    * If {@link #maxSize} is set to {@link Integer#MAX_VALUE}, then the size of
338    * the pool is unbounded. Otherwise, it caps the number of resources in this
339    * pool to the (non-zero positive) value specified in {@link #maxSize}.
340    * </p>
341    *
342    * @param <R>
343    *          the type of the resource
344    *
345    */
346   @SuppressWarnings("serial")
347   class RoundRobinPool<R> extends CopyOnWriteArrayList<R> implements Pool<R> {
348     private int maxSize;
349     private int nextResource = 0;
350 
351     public RoundRobinPool(int maxSize) {
352       this.maxSize = maxSize;
353     }
354 
355     @Override
356     public R put(R resource) {
357       if (super.size() < maxSize) {
358         add(resource);
359       }
360       return null;
361     }
362 
363     @Override
364     public R get() {
365       if (super.size() < maxSize) {
366         return null;
367       }
368       nextResource %= super.size();
369       R resource = get(nextResource++);
370       return resource;
371     }
372 
373     @Override
374     public Collection<R> values() {
375       return this;
376     }
377 
378   }
379 
380   /**
381    * The <code>ThreadLocalPool</code> represents a {@link PoolMap.Pool} that
382    * builds on the {@link ThreadLocal} class. It essentially binds the resource
383    * to the thread from which it is accessed.
384    *
385    * <p>
386    * Note that the size of the pool is essentially bounded by the number of threads
387    * that add resources to this pool.
388    * </p>
389    *
390    * @param <R>
391    *          the type of the resource
392    */
393   static class ThreadLocalPool<R> extends ThreadLocal<R> implements Pool<R> {
394     private static final Map<ThreadLocalPool<?>, AtomicInteger> poolSizes = new HashMap<ThreadLocalPool<?>, AtomicInteger>();
395 
396     public ThreadLocalPool() {
397     }
398 
399     @Override
400     public R put(R resource) {
401       R previousResource = get();
402       if (previousResource == null) {
403         AtomicInteger poolSize = poolSizes.get(this);
404         if (poolSize == null) {
405           poolSizes.put(this, poolSize = new AtomicInteger(0));
406         }
407         poolSize.incrementAndGet();
408       }
409       this.set(resource);
410       return previousResource;
411     }
412 
413     @Override
414     public void remove() {
415       super.remove();
416       AtomicInteger poolSize = poolSizes.get(this);
417       if (poolSize != null) {
418         poolSize.decrementAndGet();
419       }
420     }
421 
422     @Override
423     public int size() {
424       AtomicInteger poolSize = poolSizes.get(this);
425       return poolSize != null ? poolSize.get() : 0;
426     }
427 
428     @Override
429     public boolean remove(R resource) {
430       R previousResource = super.get();
431       if (resource != null && resource.equals(previousResource)) {
432         remove();
433         return true;
434       } else {
435         return false;
436       }
437     }
438 
439     @Override
440     public void clear() {
441       super.remove();
442     }
443 
444     @Override
445     public Collection<R> values() {
446       List<R> values = new ArrayList<R>();
447       values.add(get());
448       return values;
449     }
450   }
451 }