HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / util / ObjectPool.java
blob30f5a4636103774483810ae93353f168f30fbe9b
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.Reference;
22 import java.lang.ref.ReferenceQueue;
23 import java.util.Objects;
24 import java.util.concurrent.ConcurrentHashMap;
25 import java.util.concurrent.ConcurrentMap;
26 import java.util.concurrent.locks.Lock;
27 import java.util.concurrent.locks.ReentrantLock;
29 import org.apache.yetus.audience.InterfaceAudience;
31 /**
32 * A thread-safe shared object pool in which object creation is expected to be lightweight, and the
33 * objects may be excessively created and discarded.
35 @InterfaceAudience.Private
36 public abstract class ObjectPool<K, V> {
37 /**
38 * An {@code ObjectFactory} object is used to create
39 * new shared objects on demand.
41 public interface ObjectFactory<K, V> {
42 /**
43 * Creates a new shared object associated with the given {@code key},
44 * identified by the {@code equals} method.
45 * This method may be simultaneously called by multiple threads
46 * with the same key, and the excessive objects are just discarded.
48 V createObject(K key);
51 protected final ReferenceQueue<V> staleRefQueue = new ReferenceQueue<>();
53 private final ObjectFactory<K, V> objectFactory;
55 /** Does not permit null keys. */
56 protected final ConcurrentMap<K, Reference<V>> referenceCache;
58 /** For preventing parallel purge */
59 private final Lock purgeLock = new ReentrantLock();
61 /**
62 * The default initial capacity,
63 * used when not otherwise specified in a constructor.
65 public static final int DEFAULT_INITIAL_CAPACITY = 16;
67 /**
68 * The default concurrency level,
69 * used when not otherwise specified in a constructor.
71 public static final int DEFAULT_CONCURRENCY_LEVEL = 16;
73 /**
74 * Creates a new pool with the default initial capacity (16)
75 * and the default concurrency level (16).
77 * @param objectFactory the factory to supply new objects on demand
79 * @throws NullPointerException if {@code objectFactory} is {@code null}
81 public ObjectPool(ObjectFactory<K, V> objectFactory) {
82 this(objectFactory, DEFAULT_INITIAL_CAPACITY, DEFAULT_CONCURRENCY_LEVEL);
85 /**
86 * Creates a new pool with the given initial capacity
87 * and the default concurrency level (16).
89 * @param objectFactory the factory to supply new objects on demand
90 * @param initialCapacity the initial capacity to keep objects in the pool
92 * @throws NullPointerException if {@code objectFactory} is {@code null}
93 * @throws IllegalArgumentException if {@code initialCapacity} is negative
95 public ObjectPool(ObjectFactory<K, V> objectFactory, int initialCapacity) {
96 this(objectFactory, initialCapacity, DEFAULT_CONCURRENCY_LEVEL);
99 /**
100 * Creates a new pool with the given initial capacity
101 * and the given concurrency level.
103 * @param objectFactory the factory to supply new objects on demand
104 * @param initialCapacity the initial capacity to keep objects in the pool
105 * @param concurrencyLevel the estimated count of concurrently accessing threads
107 * @throws NullPointerException if {@code objectFactory} is {@code null}
108 * @throws IllegalArgumentException if {@code initialCapacity} is negative or
109 * {@code concurrencyLevel} is non-positive
111 public ObjectPool(
112 ObjectFactory<K, V> objectFactory,
113 int initialCapacity,
114 int concurrencyLevel) {
116 this.objectFactory = Objects.requireNonNull(objectFactory, "Object factory cannot be null");
118 this.referenceCache =
119 new ConcurrentHashMap<K, Reference<V>>(initialCapacity, 0.75f, concurrencyLevel);
123 * Removes stale references of shared objects from the pool. References newly becoming stale may
124 * still remain.
125 * <p/>
126 * The implementation of this method is expected to be lightweight when there is no stale
127 * reference with the Oracle (Sun) implementation of {@code ReferenceQueue}, because
128 * {@code ReferenceQueue.poll} just checks a volatile instance variable in {@code ReferenceQueue}.
130 public void purge() {
131 if (purgeLock.tryLock()) {// no parallel purge
132 try {
133 while (true) {
134 @SuppressWarnings("unchecked")
135 Reference<V> ref = (Reference<V>) staleRefQueue.poll();
136 if (ref == null) {
137 break;
139 referenceCache.remove(getReferenceKey(ref), ref);
141 } finally {
142 purgeLock.unlock();
148 * Create a reference associated with the given object
149 * @param key the key to store in the reference
150 * @param obj the object to associate with
151 * @return the reference instance
153 public abstract Reference<V> createReference(K key, V obj);
156 * Get key of the given reference
157 * @param ref The reference
158 * @return key of the reference
160 public abstract K getReferenceKey(Reference<V> ref);
163 * Returns a shared object associated with the given {@code key},
164 * which is identified by the {@code equals} method.
165 * @throws NullPointerException if {@code key} is {@code null}
167 public V get(K key) {
168 Reference<V> ref = referenceCache.get(Objects.requireNonNull(key));
169 if (ref != null) {
170 V obj = ref.get();
171 if (obj != null) {
172 return obj;
174 referenceCache.remove(key, ref);
177 V newObj = objectFactory.createObject(key);
178 Reference<V> newRef = createReference(key, newObj);
179 while (true) {
180 Reference<V> existingRef = referenceCache.putIfAbsent(key, newRef);
181 if (existingRef == null) {
182 return newObj;
185 V existingObject = existingRef.get();
186 if (existingObject != null) {
187 return existingObject;
189 referenceCache.remove(key, existingRef);
194 * Returns an estimated count of objects kept in the pool.
195 * This also counts stale references,
196 * and you might want to call {@link #purge()} beforehand.
198 public int size() {
199 return referenceCache.size();