View Javadoc

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
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
17   */
18  
19  package org.apache.hadoop.hbase.util;
20  
21  import java.util.concurrent.locks.Condition;
22  import java.util.concurrent.locks.ReentrantLock;
23  import java.util.concurrent.BlockingQueue;
24  import java.util.concurrent.TimeUnit;
25  import java.util.Collection;
26  import java.util.Comparator;
27  import java.util.Iterator;
28  import java.util.AbstractQueue;
29  
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.classification.InterfaceStability;
32  
33  
34  /**
35   * A generic bounded blocking Priority-Queue.
36   *
37   * The elements of the priority queue are ordered according to the Comparator
38   * provided at queue construction time.
39   *
40   * If multiple elements have the same priority this queue orders them in
41   * FIFO (first-in-first-out) manner.
42   * The head of this queue is the least element with respect to the specified
43   * ordering. If multiple elements are tied for least value, the head is the
44   * first one inserted.
45   * The queue retrieval operations poll, remove, peek, and element access the
46   * element at the head of the queue.
47   */
48  @InterfaceAudience.Private
49  @InterfaceStability.Stable
50  public class BoundedPriorityBlockingQueue<E> extends AbstractQueue<E> implements BlockingQueue<E> {
51    private static class PriorityQueue<E> {
52      private final Comparator<? super E> comparator;
53      private final E[] objects;
54  
55      private int head = 0;
56      private int tail = 0;
57  
58      @SuppressWarnings("unchecked")
59      public PriorityQueue(int capacity, Comparator<? super E> comparator) {
60        this.objects = (E[])new Object[capacity];
61        this.comparator = comparator;
62      }
63  
64      public void add(E elem) {
65        if (tail == objects.length) {
66          // shift down |-----AAAAAAA|
67          tail -= head;
68          System.arraycopy(objects, head, objects, 0, tail);
69          head = 0;
70        }
71  
72        if (tail == head || comparator.compare(objects[tail - 1], elem) <= 0) {
73          // Append
74          objects[tail++] = elem;
75        } else if (head > 0 && comparator.compare(objects[head], elem) > 0) {
76          // Prepend
77          objects[--head] = elem;
78        } else {
79          // Insert in the middle
80          int index = upperBound(head, tail - 1, elem);
81          System.arraycopy(objects, index, objects, index + 1, tail - index);
82          objects[index] = elem;
83          tail++;
84        }
85      }
86  
87      public E peek() {
88        return (head != tail) ? objects[head] : null;
89      }
90  
91      public E poll() {
92        E elem = objects[head];
93        head = (head + 1) % objects.length;
94        if (head == 0) tail = 0;
95        return elem;
96      }
97  
98      public int size() {
99        return tail - head;
100     }
101 
102     public Comparator<? super E> comparator() {
103       return this.comparator;
104     }
105 
106     public boolean contains(Object o) {
107       for (int i = head; i < tail; ++i) {
108         if (objects[i] == o) {
109           return true;
110         }
111       }
112       return false;
113     }
114 
115     public int remainingCapacity() {
116       return this.objects.length - (tail - head);
117     }
118 
119     private int upperBound(int start, int end, E key) {
120       while (start < end) {
121         int mid = (start + end) >>> 1;
122         E mitem = objects[mid];
123         int cmp = comparator.compare(mitem, key);
124         if (cmp > 0) {
125           end = mid;
126         } else {
127           start = mid + 1;
128         }
129       }
130       return start;
131     }
132   }
133 
134 
135   // Lock used for all operations
136   private final ReentrantLock lock = new ReentrantLock();
137 
138   // Condition for blocking when empty
139   private final Condition notEmpty = lock.newCondition();
140 
141   // Wait queue for waiting puts
142   private final Condition notFull = lock.newCondition();
143 
144   private final PriorityQueue<E> queue;
145 
146   /**
147    * Creates a PriorityQueue with the specified capacity that orders its
148    * elements according to the specified comparator.
149    * @param capacity the capacity of this queue
150    * @param comparator the comparator that will be used to order this priority queue
151    */
152   public BoundedPriorityBlockingQueue(int capacity,
153       Comparator<? super E> comparator) {
154     this.queue = new PriorityQueue<E>(capacity, comparator);
155   }
156 
157   public boolean offer(E e) {
158     if (e == null) throw new NullPointerException();
159 
160     lock.lock();
161     try {
162       if (queue.remainingCapacity() > 0) {
163         this.queue.add(e);
164         notEmpty.signal();
165         return true;
166       }
167     } finally {
168       lock.unlock();
169     }
170     return false;
171   }
172 
173   public void put(E e) throws InterruptedException {
174     if (e == null) throw new NullPointerException();
175 
176     lock.lock();
177     try {
178       while (queue.remainingCapacity() == 0) {
179         notFull.await();
180       }
181       this.queue.add(e);
182       notEmpty.signal();
183     } finally {
184       lock.unlock();
185     }
186   }
187 
188   public boolean offer(E e, long timeout, TimeUnit unit)
189       throws InterruptedException {
190     if (e == null) throw new NullPointerException();
191     long nanos = unit.toNanos(timeout);
192 
193     lock.lockInterruptibly();
194     try {
195       while (queue.remainingCapacity() == 0) {
196         if (nanos <= 0)
197           return false;
198         nanos = notFull.awaitNanos(nanos);
199       }
200       this.queue.add(e);
201       notEmpty.signal();
202     } finally {
203       lock.unlock();
204     }
205     return true;
206   }
207 
208   public E take() throws InterruptedException {
209     E result = null;
210     lock.lockInterruptibly();
211     try {
212       while (queue.size() == 0) {
213         notEmpty.await();
214       }
215       result = queue.poll();
216       notFull.signal();
217     } finally {
218       lock.unlock();
219     }
220     return result;
221   }
222 
223   public E poll() {
224     E result = null;
225     lock.lock();
226     try {
227       if (queue.size() > 0) {
228         result = queue.poll();
229         notFull.signal();
230       }
231     } finally {
232       lock.unlock();
233     }
234     return result;
235   }
236 
237   public E poll(long timeout, TimeUnit unit)
238       throws InterruptedException {
239     long nanos = unit.toNanos(timeout);
240     lock.lockInterruptibly();
241     E result = null;
242     try {
243       while (queue.size() == 0 && nanos > 0) {
244         nanos = notEmpty.awaitNanos(nanos);
245       }
246       if (queue.size() > 0) {
247         result = queue.poll();
248       }
249       notFull.signal();
250     } finally {
251       lock.unlock();
252     }
253     return result;
254   }
255 
256   public E peek() {
257     lock.lock();
258     try {
259       return queue.peek();
260     } finally {
261       lock.unlock();
262     }
263   }
264 
265   public int size() {
266     lock.lock();
267     try {
268       return queue.size();
269     } finally {
270       lock.unlock();
271     }
272   }
273 
274   public Iterator<E> iterator() {
275     throw new UnsupportedOperationException();
276   }
277 
278   public Comparator<? super E> comparator() {
279     return queue.comparator();
280   }
281 
282   public int remainingCapacity() {
283     lock.lock();
284     try {
285       return queue.remainingCapacity();
286     } finally {
287       lock.unlock();
288     }
289   }
290 
291   public boolean remove(Object o) {
292     throw new UnsupportedOperationException();
293   }
294 
295   public boolean contains(Object o) {
296     lock.lock();
297     try {
298       return queue.contains(o);
299     } finally {
300       lock.unlock();
301     }
302   }
303 
304   public int drainTo(Collection<? super E> c) {
305     return drainTo(c, Integer.MAX_VALUE);
306   }
307 
308   public int drainTo(Collection<? super E> c, int maxElements) {
309     if (c == null)
310         throw new NullPointerException();
311     if (c == this)
312         throw new IllegalArgumentException();
313     if (maxElements <= 0)
314         return 0;
315     lock.lock();
316     try {
317       int n = Math.min(queue.size(), maxElements);
318       for (int i = 0; i < n; ++i) {
319         c.add(queue.poll());
320       }
321       return n;
322     } finally {
323       lock.unlock();
324     }
325   }
326 }