HBASE-17532 Replaced explicit type with diamond operator
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / util / WeakObjectPool.java
blob478864b338b421212ef51b4ef31bf5e7fb72f02e
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
19 package org.apache.hadoop.hbase.util;
21 import java.lang.ref.ReferenceQueue;
22 import java.lang.ref.WeakReference;
23 import java.util.concurrent.ConcurrentHashMap;
24 import java.util.concurrent.ConcurrentMap;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 /**
29 * A {@code WeakReference} based shared object pool.
30 * The objects are kept in weak references and
31 * associated with keys which are identified by the {@code equals} method.
32 * The objects are created by {@link ObjectFactory} on demand.
33 * The object creation is expected to be lightweight,
34 * and the objects may be excessively created and discarded.
35 * Thread safe.
37 @InterfaceAudience.Private
38 public class WeakObjectPool<K, V> {
39 /**
40 * An {@code ObjectFactory} object is used to create
41 * new shared objects on demand.
43 public interface ObjectFactory<K, V> {
44 /**
45 * Creates a new shared object associated with the given {@code key},
46 * identified by the {@code equals} method.
47 * This method may be simultaneously called by multiple threads
48 * with the same key, and the excessive objects are just discarded.
50 V createObject(K key);
53 private final ReferenceQueue<V> staleRefQueue = new ReferenceQueue<>();
55 private class ObjectReference extends WeakReference<V> {
56 final K key;
58 ObjectReference(K key, V obj) {
59 super(obj, staleRefQueue);
60 this.key = key;
64 private final ObjectFactory<K, V> objectFactory;
66 /** Does not permit null keys. */
67 private final ConcurrentMap<K, ObjectReference> referenceCache;
69 /**
70 * The default initial capacity,
71 * used when not otherwise specified in a constructor.
73 public static final int DEFAULT_INITIAL_CAPACITY = 16;
75 /**
76 * The default concurrency level,
77 * used when not otherwise specified in a constructor.
79 public static final int DEFAULT_CONCURRENCY_LEVEL = 16;
81 /**
82 * Creates a new pool with the default initial capacity (16)
83 * and the default concurrency level (16).
85 * @param objectFactory the factory to supply new objects on demand
87 * @throws NullPointerException if {@code objectFactory} is null
89 public WeakObjectPool(ObjectFactory<K, V> objectFactory) {
90 this(objectFactory, DEFAULT_INITIAL_CAPACITY, DEFAULT_CONCURRENCY_LEVEL);
93 /**
94 * Creates a new pool with the given initial capacity
95 * and the default concurrency level (16).
97 * @param objectFactory the factory to supply new objects on demand
98 * @param initialCapacity the initial capacity to keep objects in the pool
100 * @throws NullPointerException if {@code objectFactory} is null
101 * @throws IllegalArgumentException if {@code initialCapacity} is negative
103 public WeakObjectPool(ObjectFactory<K, V> objectFactory, int initialCapacity) {
104 this(objectFactory, initialCapacity, DEFAULT_CONCURRENCY_LEVEL);
108 * Creates a new pool with the given initial capacity
109 * and the given concurrency level.
111 * @param objectFactory the factory to supply new objects on demand
112 * @param initialCapacity the initial capacity to keep objects in the pool
113 * @param concurrencyLevel the estimated count of concurrently accessing threads
115 * @throws NullPointerException if {@code objectFactory} is null
116 * @throws IllegalArgumentException if {@code initialCapacity} is negative or
117 * {@code concurrencyLevel} is non-positive
119 public WeakObjectPool(
120 ObjectFactory<K, V> objectFactory,
121 int initialCapacity,
122 int concurrencyLevel) {
124 if (objectFactory == null) {
125 throw new NullPointerException();
127 this.objectFactory = objectFactory;
129 this.referenceCache = new ConcurrentHashMap<>(initialCapacity, 0.75f, concurrencyLevel);
130 // 0.75f is the default load factor threshold of ConcurrentHashMap.
134 * Removes stale references of shared objects from the pool.
135 * References newly becoming stale may still remain.
136 * The implementation of this method is expected to be lightweight
137 * when there is no stale reference.
139 public void purge() {
140 // This method is lightweight while there is no stale reference
141 // with the Oracle (Sun) implementation of {@code ReferenceQueue},
142 // because {@code ReferenceQueue.poll} just checks a volatile instance
143 // variable in {@code ReferenceQueue}.
145 while (true) {
146 @SuppressWarnings("unchecked")
147 ObjectReference ref = (ObjectReference)staleRefQueue.poll();
148 if (ref == null) {
149 break;
151 referenceCache.remove(ref.key, ref);
156 * Returns a shared object associated with the given {@code key},
157 * which is identified by the {@code equals} method.
158 * @throws NullPointerException if {@code key} is null
160 public V get(K key) {
161 ObjectReference ref = referenceCache.get(key);
162 if (ref != null) {
163 V obj = ref.get();
164 if (obj != null) {
165 return obj;
167 referenceCache.remove(key, ref);
170 V newObj = objectFactory.createObject(key);
171 ObjectReference newRef = new ObjectReference(key, newObj);
172 while (true) {
173 ObjectReference existingRef = referenceCache.putIfAbsent(key, newRef);
174 if (existingRef == null) {
175 return newObj;
178 V existingObject = existingRef.get();
179 if (existingObject != null) {
180 return existingObject;
182 referenceCache.remove(key, existingRef);
187 * Returns an estimated count of objects kept in the pool.
188 * This also counts stale references,
189 * and you might want to call {@link #purge()} beforehand.
191 public int size() {
192 return referenceCache.size();