-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
RegionNormalizerWorkQueue.java
195 lines (182 loc) · 5.51 KB
/
RegionNormalizerWorkQueue.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.normalizer;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is an
* ordered collection class that has the following properties:
* <ul>
* <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
* <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
* <li>Work is retrieved on a FIFO policy.</li>
* <li>Work retrieval blocks the calling thread until new work is available, as a
* {@link BlockingQueue}.</li>
* <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
* </ul>
*/
@InterfaceAudience.Private
class RegionNormalizerWorkQueue<E> {
/** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
private LinkedHashSet<E> delegate;
/** Lock for puts and takes **/
private final ReentrantReadWriteLock lock;
/** Wait queue for waiting takes */
private final Condition notEmpty;
RegionNormalizerWorkQueue() {
delegate = new LinkedHashSet<>();
lock = new ReentrantReadWriteLock();
notEmpty = lock.writeLock().newCondition();
}
/**
* Inserts the specified element at the tail of the queue, if it's not already present.
* @param e the element to add
*/
public void put(E e) {
if (e == null) {
throw new NullPointerException();
}
lock.writeLock().lock();
try {
delegate.add(e);
if (!delegate.isEmpty()) {
notEmpty.signal();
}
} finally {
lock.writeLock().unlock();
}
}
/**
* Inserts the specified element at the head of the queue.
* @param e the element to add
*/
public void putFirst(E e) {
if (e == null) {
throw new NullPointerException();
}
putAllFirst(Collections.singleton(e));
}
/**
* Inserts the specified elements at the tail of the queue. Any elements already present in the
* queue are ignored.
* @param c the elements to add
*/
public void putAll(Collection<? extends E> c) {
if (c == null) {
throw new NullPointerException();
}
lock.writeLock().lock();
try {
delegate.addAll(c);
if (!delegate.isEmpty()) {
notEmpty.signal();
}
} finally {
lock.writeLock().unlock();
}
}
/**
* Inserts the specified elements at the head of the queue.
* @param c the elements to add
*/
public void putAllFirst(Collection<? extends E> c) {
if (c == null) {
throw new NullPointerException();
}
lock.writeLock().lock();
try {
final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
copy.addAll(c);
copy.addAll(delegate);
delegate = copy;
if (!delegate.isEmpty()) {
notEmpty.signal();
}
} finally {
lock.writeLock().unlock();
}
}
/**
* Retrieves and removes the head of this queue, waiting if necessary until an element becomes
* available.
* @return the head of this queue
* @throws InterruptedException if interrupted while waiting
*/
public E take() throws InterruptedException {
E x;
// Take a write lock. If the delegate's queue is empty we need it to await(), which will
// drop the lock, then reacquire it; or if the queue is not empty we will use an iterator
// to mutate the head.
lock.writeLock().lockInterruptibly();
try {
while (delegate.isEmpty()) {
notEmpty.await(); // await drops the lock, then reacquires it
}
final Iterator<E> iter = delegate.iterator();
x = iter.next();
iter.remove();
if (!delegate.isEmpty()) {
notEmpty.signal();
}
} finally {
lock.writeLock().unlock();
}
return x;
}
/**
* Atomically removes all of the elements from this queue. The queue will be empty after this call
* returns.
*/
public void clear() {
lock.writeLock().lock();
try {
delegate.clear();
} finally {
lock.writeLock().unlock();
}
}
/**
* Returns the number of elements in this queue.
* @return the number of elements in this queue
*/
public int size() {
lock.readLock().lock();
try {
return delegate.size();
} finally {
lock.readLock().unlock();
}
}
@Override
public String toString() {
lock.readLock().lock();
try {
return delegate.toString();
} finally {
lock.readLock().unlock();
}
}
}