Skip to content

Commit 0bb47f5

Browse files
authored
Revert "HBASE-27064 CME in TestRegionNormalizerWorkQueue (apache#4468)"
This reverts commit a93ea9e.
1 parent 193e349 commit 0bb47f5

File tree

1 file changed

+75
-32
lines changed

1 file changed

+75
-32
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java

Lines changed: 75 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@
2525
import java.util.Set;
2626
import java.util.concurrent.BlockingQueue;
2727
import java.util.concurrent.locks.Condition;
28-
import java.util.concurrent.locks.ReentrantReadWriteLock;
28+
import java.util.concurrent.locks.ReentrantLock;
2929
import org.apache.yetus.audience.InterfaceAudience;
3030

3131
/**
@@ -39,22 +39,62 @@
3939
* {@link BlockingQueue}.</li>
4040
* <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
4141
* </ul>
42+
* Assumes low-frequency and low-parallelism concurrent access, so protects state using a simplistic
43+
* synchronization strategy.
4244
*/
4345
@InterfaceAudience.Private
4446
class RegionNormalizerWorkQueue<E> {
4547

4648
/** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
4749
private LinkedHashSet<E> delegate;
4850

49-
/** Lock for puts and takes **/
50-
private final ReentrantReadWriteLock lock;
51+
// the locking structure used here follows the example found in LinkedBlockingQueue. The
52+
// difference is that our locks guard access to `delegate` rather than the head node.
53+
54+
/** Lock held by take, poll, etc */
55+
private final ReentrantLock takeLock;
56+
5157
/** Wait queue for waiting takes */
5258
private final Condition notEmpty;
5359

60+
/** Lock held by put, offer, etc */
61+
private final ReentrantLock putLock;
62+
5463
RegionNormalizerWorkQueue() {
5564
delegate = new LinkedHashSet<>();
56-
lock = new ReentrantReadWriteLock();
57-
notEmpty = lock.writeLock().newCondition();
65+
takeLock = new ReentrantLock();
66+
notEmpty = takeLock.newCondition();
67+
putLock = new ReentrantLock();
68+
}
69+
70+
/**
71+
* Signals a waiting take. Called only from put/offer (which do not otherwise ordinarily lock
72+
* takeLock.)
73+
*/
74+
private void signalNotEmpty() {
75+
final ReentrantLock takeLock = this.takeLock;
76+
takeLock.lock();
77+
try {
78+
notEmpty.signal();
79+
} finally {
80+
takeLock.unlock();
81+
}
82+
}
83+
84+
/**
85+
* Locks to prevent both puts and takes.
86+
*/
87+
private void fullyLock() {
88+
putLock.lock();
89+
takeLock.lock();
90+
}
91+
92+
/**
93+
* Unlocks to allow both puts and takes.
94+
*/
95+
private void fullyUnlock() {
96+
takeLock.unlock();
97+
putLock.unlock();
5898
}
5999

60100
/**
@@ -65,14 +105,16 @@ public void put(E e) {
65105
if (e == null) {
66106
throw new NullPointerException();
67107
}
68-
lock.writeLock().lock();
108+
109+
putLock.lock();
69110
try {
70111
delegate.add(e);
71-
if (!delegate.isEmpty()) {
72-
notEmpty.signal();
73-
}
74112
} finally {
75-
lock.writeLock().unlock();
113+
putLock.unlock();
114+
}
115+
116+
if (!delegate.isEmpty()) {
117+
signalNotEmpty();
76118
}
77119
}
78120

@@ -96,14 +138,16 @@ public void putAll(Collection<? extends E> c) {
96138
if (c == null) {
97139
throw new NullPointerException();
98140
}
99-
lock.writeLock().lock();
141+
142+
putLock.lock();
100143
try {
101144
delegate.addAll(c);
102-
if (!delegate.isEmpty()) {
103-
notEmpty.signal();
104-
}
105145
} finally {
106-
lock.writeLock().unlock();
146+
putLock.unlock();
147+
}
148+
149+
if (!delegate.isEmpty()) {
150+
signalNotEmpty();
107151
}
108152
}
109153

@@ -115,17 +159,19 @@ public void putAllFirst(Collection<? extends E> c) {
115159
if (c == null) {
116160
throw new NullPointerException();
117161
}
118-
lock.writeLock().lock();
162+
163+
fullyLock();
119164
try {
120165
final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
121166
copy.addAll(c);
122167
copy.addAll(delegate);
123168
delegate = copy;
124-
if (!delegate.isEmpty()) {
125-
notEmpty.signal();
126-
}
127169
} finally {
128-
lock.writeLock().unlock();
170+
fullyUnlock();
171+
}
172+
173+
if (!delegate.isEmpty()) {
174+
signalNotEmpty();
129175
}
130176
}
131177

@@ -137,13 +183,10 @@ public void putAllFirst(Collection<? extends E> c) {
137183
*/
138184
public E take() throws InterruptedException {
139185
E x;
140-
// Take a write lock. If the delegate's queue is empty we need it to await(), which will
141-
// drop the lock, then reacquire it; or if the queue is not empty we will use an iterator
142-
// to mutate the head.
143-
lock.writeLock().lockInterruptibly();
186+
takeLock.lockInterruptibly();
144187
try {
145188
while (delegate.isEmpty()) {
146-
notEmpty.await(); // await drops the lock, then reacquires it
189+
notEmpty.await();
147190
}
148191
final Iterator<E> iter = delegate.iterator();
149192
x = iter.next();
@@ -152,7 +195,7 @@ public E take() throws InterruptedException {
152195
notEmpty.signal();
153196
}
154197
} finally {
155-
lock.writeLock().unlock();
198+
takeLock.unlock();
156199
}
157200
return x;
158201
}
@@ -162,11 +205,11 @@ public E take() throws InterruptedException {
162205
* returns.
163206
*/
164207
public void clear() {
165-
lock.writeLock().lock();
208+
putLock.lock();
166209
try {
167210
delegate.clear();
168211
} finally {
169-
lock.writeLock().unlock();
212+
putLock.unlock();
170213
}
171214
}
172215

@@ -175,21 +218,21 @@ public void clear() {
175218
* @return the number of elements in this queue
176219
*/
177220
public int size() {
178-
lock.readLock().lock();
221+
takeLock.lock();
179222
try {
180223
return delegate.size();
181224
} finally {
182-
lock.readLock().unlock();
225+
takeLock.unlock();
183226
}
184227
}
185228

186229
@Override
187230
public String toString() {
188-
lock.readLock().lock();
231+
takeLock.lock();
189232
try {
190233
return delegate.toString();
191234
} finally {
192-
lock.readLock().unlock();
235+
takeLock.unlock();
193236
}
194237
}
195238
}

0 commit comments

Comments
 (0)