HBASE-24163 MOB compactor implementations should use format specifiers when calling...
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / util / KeyLocker.java
blobb4c150b6573c4639d4f01fa052c2dcd64a1301e5
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.util.Arrays;
22 import java.util.LinkedHashMap;
23 import java.util.Map;
24 import java.util.Set;
25 import java.util.concurrent.locks.Lock;
26 import java.util.concurrent.locks.ReentrantLock;
28 import org.apache.yetus.audience.InterfaceAudience;
30 /**
31 * A utility class to manage a set of locks. Each lock is identified by a String which serves
32 * as a key. Typical usage is: <pre>
33 * class Example {
34 * private final static KeyLocker&lt;String&gt; locker = new Locker&lt;String&gt;();
35 * public void foo(String s){
36 * Lock lock = locker.acquireLock(s);
37 * try {
38 * // whatever
39 * }finally{
40 * lock.unlock();
41 * }
42 * }
43 * }
44 * </pre>
46 @InterfaceAudience.Private
47 public class KeyLocker<K> {
48 // The number of lock we want to easily support. It's not a maximum.
49 private static final int NB_CONCURRENT_LOCKS = 1000;
51 private final WeakObjectPool<K, ReentrantLock> lockPool =
52 new WeakObjectPool<>(
53 new ObjectPool.ObjectFactory<K, ReentrantLock>() {
54 @Override
55 public ReentrantLock createObject(K key) {
56 return new ReentrantLock();
59 NB_CONCURRENT_LOCKS);
61 /**
62 * Return a lock for the given key. The lock is already locked.
64 * @param key
66 public ReentrantLock acquireLock(K key) {
67 if (key == null) throw new IllegalArgumentException("key must not be null");
69 lockPool.purge();
70 ReentrantLock lock = lockPool.get(key);
72 lock.lock();
73 return lock;
76 /**
77 * Acquire locks for a set of keys. The keys will be
78 * sorted internally to avoid possible deadlock.
80 * @throws ClassCastException if the given {@code keys}
81 * contains elements that are not mutually comparable
83 public Map<K, Lock> acquireLocks(Set<? extends K> keys) {
84 Object[] keyArray = keys.toArray();
85 Arrays.sort(keyArray);
87 lockPool.purge();
88 Map<K, Lock> locks = new LinkedHashMap<>(keyArray.length);
89 for (Object o : keyArray) {
90 @SuppressWarnings("unchecked")
91 K key = (K)o;
92 ReentrantLock lock = lockPool.get(key);
93 locks.put(key, lock);
96 for (Lock lock : locks.values()) {
97 lock.lock();
99 return locks;