HBASE-21843 RegionGroupingProvider breaks the meta wal file name pattern which may...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / util / IdReadWriteLock.java
blob852f5c0dd328953f8c675880e901b81632adb383
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.lang.ref.Reference;
22 import java.util.concurrent.locks.ReentrantReadWriteLock;
24 import org.apache.yetus.audience.InterfaceAudience;
26 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
28 /**
29 * Allows multiple concurrent clients to lock on a numeric id with ReentrantReadWriteLock. The
30 * intended usage for read lock is as follows:
32 * <pre>
33 * ReentrantReadWriteLock lock = idReadWriteLock.getLock(id);
34 * try {
35 * lock.readLock().lock();
36 * // User code.
37 * } finally {
38 * lock.readLock().unlock();
39 * }
40 * </pre>
42 * For write lock, use lock.writeLock()
44 @InterfaceAudience.Private
45 public class IdReadWriteLock<T> {
46 // The number of lock we want to easily support. It's not a maximum.
47 private static final int NB_CONCURRENT_LOCKS = 1000;
48 /**
49 * The pool to get entry from, entries are mapped by {@link Reference} and will be automatically
50 * garbage-collected by JVM
52 private final ObjectPool<T, ReentrantReadWriteLock> lockPool;
53 private final ReferenceType refType;
55 public IdReadWriteLock() {
56 this(ReferenceType.WEAK);
59 /**
60 * Constructor of IdReadWriteLock
61 * @param referenceType type of the reference used in lock pool, {@link ReferenceType#WEAK} by
62 * default. Use {@link ReferenceType#SOFT} if the key set is limited and the locks will
63 * be reused with a high frequency
65 public IdReadWriteLock(ReferenceType referenceType) {
66 this.refType = referenceType;
67 switch (referenceType) {
68 case SOFT:
69 lockPool = new SoftObjectPool<>(new ObjectPool.ObjectFactory<T, ReentrantReadWriteLock>() {
70 @Override
71 public ReentrantReadWriteLock createObject(T id) {
72 return new ReentrantReadWriteLock();
74 }, NB_CONCURRENT_LOCKS);
75 break;
76 case WEAK:
77 default:
78 lockPool = new WeakObjectPool<>(new ObjectPool.ObjectFactory<T, ReentrantReadWriteLock>() {
79 @Override
80 public ReentrantReadWriteLock createObject(T id) {
81 return new ReentrantReadWriteLock();
83 }, NB_CONCURRENT_LOCKS);
87 public static enum ReferenceType {
88 WEAK, SOFT
91 /**
92 * Get the ReentrantReadWriteLock corresponding to the given id
93 * @param id an arbitrary number to identify the lock
95 public ReentrantReadWriteLock getLock(T id) {
96 lockPool.purge();
97 ReentrantReadWriteLock readWriteLock = lockPool.get(id);
98 return readWriteLock;
101 /** For testing */
102 @VisibleForTesting
103 int purgeAndGetEntryPoolSize() {
104 gc();
105 Threads.sleep(200);
106 lockPool.purge();
107 return lockPool.size();
110 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DM_GC", justification="Intentional")
111 private void gc() {
112 System.gc();
115 @VisibleForTesting
116 public void waitForWaiters(T id, int numWaiters) throws InterruptedException {
117 for (ReentrantReadWriteLock readWriteLock;;) {
118 readWriteLock = lockPool.get(id);
119 if (readWriteLock != null) {
120 synchronized (readWriteLock) {
121 if (readWriteLock.getQueueLength() >= numWaiters) {
122 return;
126 Thread.sleep(50);
130 @VisibleForTesting
131 public ReferenceType getReferenceType() {
132 return this.refType;