HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / util / PoolMap.java
blob057cb7e37555802a7719b8e2bcc10e4bf903c851
1 /**
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
11 * http://www.apache.org/licenses/LICENSE-2.0
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.
19 package org.apache.hadoop.hbase.util;
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Collection;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Locale;
27 import java.util.Map;
28 import java.util.Objects;
30 import org.apache.yetus.audience.InterfaceAudience;
32 /**
34 * The <code>PoolMap</code> maps a key to a collection of values, the elements
35 * of which are managed by a pool. In effect, that collection acts as a shared
36 * pool of resources, access to which is closely controlled as per the semantics
37 * of the pool.
39 * <p>
40 * In case the size of the pool is set to a non-zero positive number, that is
41 * used to cap the number of resources that a pool may contain for any given
42 * key. A size of {@link Integer#MAX_VALUE} is interpreted as an unbounded pool.
43 * </p>
45 * <p>
46 * PoolMap is thread-safe. It does not remove elements automatically. Unused resources
47 * must be closed and removed explicitly.
48 * </p>
50 * @param <K>
51 * the type of the key to the resource
52 * @param <V>
53 * the type of the resource being pooled
55 @InterfaceAudience.Private
56 public class PoolMap<K, V> {
57 private final Map<K, Pool<V>> pools;
58 private final PoolType poolType;
59 private final int poolMaxSize;
61 public PoolMap(PoolType poolType, int poolMaxSize) {
62 pools = new HashMap<>();
63 this.poolType = poolType;
64 this.poolMaxSize = poolMaxSize;
67 public V getOrCreate(K key, PoolResourceSupplier<V> supplier) throws IOException {
68 synchronized (pools) {
69 Pool<V> pool = pools.get(key);
71 if (pool == null) {
72 pool = createPool();
73 pools.put(key, pool);
76 try {
77 return pool.getOrCreate(supplier);
78 } catch (IOException | RuntimeException | Error e) {
79 if (pool.size() == 0) {
80 pools.remove(key);
83 throw e;
87 public boolean remove(K key, V value) {
88 synchronized (pools) {
89 Pool<V> pool = pools.get(key);
91 if (pool == null) {
92 return false;
95 boolean removed = pool.remove(value);
97 if (removed && pool.size() == 0) {
98 pools.remove(key);
101 return removed;
105 public List<V> values() {
106 List<V> values = new ArrayList<>();
108 synchronized (pools) {
109 for (Pool<V> pool : pools.values()) {
110 Collection<V> poolValues = pool.values();
111 if (poolValues != null) {
112 values.addAll(poolValues);
117 return values;
120 public void clear() {
121 synchronized (pools) {
122 for (Pool<V> pool : pools.values()) {
123 pool.clear();
126 pools.clear();
130 public interface PoolResourceSupplier<R> {
131 R get() throws IOException;
134 protected static <V> V createResource(PoolResourceSupplier<V> supplier) throws IOException {
135 V resource = supplier.get();
136 return Objects.requireNonNull(resource, "resource cannot be null.");
139 protected interface Pool<R> {
140 R getOrCreate(PoolResourceSupplier<R> supplier) throws IOException;
142 boolean remove(R resource);
144 void clear();
146 Collection<R> values();
148 int size();
151 public enum PoolType {
152 ThreadLocal, RoundRobin;
154 public static PoolType valueOf(String poolTypeName, PoolType defaultPoolType) {
155 PoolType poolType = PoolType.fuzzyMatch(poolTypeName);
156 return (poolType != null) ? poolType : defaultPoolType;
159 public static String fuzzyNormalize(String name) {
160 return name != null ? name.replaceAll("-", "").trim().toLowerCase(Locale.ROOT) : "";
163 public static PoolType fuzzyMatch(String name) {
164 for (PoolType poolType : values()) {
165 if (fuzzyNormalize(name).equals(fuzzyNormalize(poolType.name()))) {
166 return poolType;
169 return null;
173 protected Pool<V> createPool() {
174 switch (poolType) {
175 case RoundRobin:
176 return new RoundRobinPool<>(poolMaxSize);
177 case ThreadLocal:
178 return new ThreadLocalPool<>();
179 default:
180 return new RoundRobinPool<>(poolMaxSize);
185 * The <code>RoundRobinPool</code> represents a {@link PoolMap.Pool}, which
186 * stores its resources in an {@link ArrayList}. It load-balances access to
187 * its resources by returning a different resource every time a given key is
188 * looked up.
190 * <p>
191 * If {@link #maxSize} is set to {@link Integer#MAX_VALUE}, then the size of
192 * the pool is unbounded. Otherwise, it caps the number of resources in this
193 * pool to the (non-zero positive) value specified in {@link #maxSize}.
194 * </p>
196 * @param <R>
197 * the type of the resource
200 @SuppressWarnings("serial")
201 static class RoundRobinPool<R> implements Pool<R> {
202 private final List<R> resources;
203 private final int maxSize;
205 private int nextIndex;
207 public RoundRobinPool(int maxSize) {
208 if (maxSize <= 0) {
209 throw new IllegalArgumentException("maxSize must be positive");
212 resources = new ArrayList<>(maxSize);
213 this.maxSize = maxSize;
216 @Override
217 public R getOrCreate(PoolResourceSupplier<R> supplier) throws IOException {
218 int size = resources.size();
219 R resource;
221 /* letting pool to grow */
222 if (size < maxSize) {
223 resource = createResource(supplier);
224 resources.add(resource);
225 } else {
226 resource = resources.get(nextIndex);
228 /* at this point size cannot be 0 */
229 nextIndex = (nextIndex + 1) % size;
232 return resource;
235 @Override
236 public boolean remove(R resource) {
237 return resources.remove(resource);
240 @Override
241 public void clear() {
242 resources.clear();
245 @Override
246 public Collection<R> values() {
247 return resources;
250 @Override
251 public int size() {
252 return resources.size();
257 * The <code>ThreadLocalPool</code> represents a {@link PoolMap.Pool} that
258 * works similarly to {@link ThreadLocal} class. It essentially binds the resource
259 * to the thread from which it is accessed. It doesn't remove resources when a thread exits,
260 * those resources must be closed manually.
262 * <p>
263 * Note that the size of the pool is essentially bounded by the number of threads
264 * that add resources to this pool.
265 * </p>
267 * @param <R>
268 * the type of the resource
270 static class ThreadLocalPool<R> implements Pool<R> {
271 private final Map<Thread, R> resources;
273 public ThreadLocalPool() {
274 resources = new HashMap<>();
277 @Override
278 public R getOrCreate(PoolResourceSupplier<R> supplier) throws IOException {
279 Thread myself = Thread.currentThread();
280 R resource = resources.get(myself);
282 if (resource == null) {
283 resource = createResource(supplier);
284 resources.put(myself, resource);
287 return resource;
290 @Override
291 public boolean remove(R resource) {
292 /* remove can be called from any thread */
293 return resources.values().remove(resource);
296 @Override
297 public int size() {
298 return resources.size();
301 @Override
302 public void clear() {
303 resources.clear();
306 @Override
307 public Collection<R> values() {
308 return resources.values();