001    /**
002     * Copyright (c) 2010 Yahoo! Inc. All rights reserved.
003     * Licensed under the Apache License, Version 2.0 (the "License");
004     * you may not use this file except in compliance with the License.
005     * You may obtain a copy of the License at
006     *
007     *   http://www.apache.org/licenses/LICENSE-2.0
008     *
009     *  Unless required by applicable law or agreed to in writing, software
010     *  distributed under the License is distributed on an "AS IS" BASIS,
011     *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
012     *  See the License for the specific language governing permissions and
013     *  limitations under the License. See accompanying LICENSE file.
014     */
015    package org.apache.oozie.util;
016    
017    import java.util.AbstractQueue;
018    import java.util.ArrayList;
019    import java.util.Arrays;
020    import java.util.Collection;
021    import java.util.ConcurrentModificationException;
022    import java.util.Iterator;
023    import java.util.List;
024    import java.util.concurrent.BlockingQueue;
025    import java.util.concurrent.DelayQueue;
026    import java.util.concurrent.Delayed;
027    import java.util.concurrent.TimeUnit;
028    import java.util.concurrent.atomic.AtomicInteger;
029    import java.util.concurrent.locks.ReentrantLock;
030    
031    /**
032     * A Queue implementation that support queuing elements into the future and priority queuing.
033     * <p/>
034     * The {@link PriorityDelayQueue} avoids starvation by raising elements priority as they age.
035     * <p/>
036     * To support queuing elements into the future, the JDK <code>DelayQueue</code> is used.
037     * <p/>
038     * To support priority queuing, an array of <code>DelayQueue</code> sub-queues is used. Elements are consumed from the
039     * higher priority sub-queues first. From a sub-queue, elements are available based on their age.
040     * <p/>
041     * To avoid starvation, there is is maximum wait time for an an element in a sub-queue, after the maximum wait time has
042     * elapsed, the element is promoted to the next higher priority sub-queue. Eventually it will reach the maximum priority
043     * sub-queue and it will be consumed when it is the oldest element in the that sub-queue.
044     * <p/>
045     * Every time an element is promoted to a higher priority sub-queue, a new maximum wait time applies.
046     * <p/>
047     * This class does not use a separate thread for anti-starvation check, instead, the check is performed on polling and
048     * seeking operations. This check is performed, the most every 1/2 second.
049     */
050    public class PriorityDelayQueue<E> extends AbstractQueue<PriorityDelayQueue.QueueElement<E>>
051            implements BlockingQueue<PriorityDelayQueue.QueueElement<E>> {
052    
053        /**
054         * Element wrapper required by the queue.
055         * <p/>
056         * This wrapper keeps track of the priority and the age of a queue element.
057         */
058        public static class QueueElement<E> implements Delayed {
059            private E element;
060            private int priority;
061            private long baseTime;
062            private boolean inQueue;
063    
064            /**
065             * Create an Element wrapper.
066             *
067             * @param element element.
068             * @param priority priority of the element.
069             * @param delay delay of the element.
070             * @param unit time unit of the delay.
071             *
072             * @throws IllegalArgumentException if the element is <tt>NULL</tt>, the priority is negative or if the delay is
073             * negative.
074             */
075            public QueueElement(E element, int priority, long delay, TimeUnit unit) {
076                if (element == null) {
077                    throw new IllegalArgumentException("element cannot be null");
078                }
079                if (priority < 0) {
080                    throw new IllegalArgumentException("priority cannot be negative, [" + element + "]");
081                }
082                if (delay < 0) {
083                    throw new IllegalArgumentException("delay cannot be negative");
084                }
085                this.element = element;
086                this.priority = priority;
087                setDelay(delay, unit);
088            }
089    
090            /**
091             * Create an Element wrapper with no delay and minimum priority.
092             *
093             * @param element element.
094             */
095            public QueueElement(E element) {
096                this(element, 0, 0, TimeUnit.MILLISECONDS);
097            }
098    
099            /**
100             * Return the element from the wrapper.
101             *
102             * @return the element.
103             */
104            public E getElement() {
105                return element;
106            }
107    
108            /**
109             * Return the priority of the element.
110             *
111             * @return the priority of the element.
112             */
113            public int getPriority() {
114                return priority;
115            }
116    
117            /**
118             * Set the delay of the element.
119             *
120             * @param delay delay of the element.
121             * @param unit time unit of the delay.
122             */
123            public void setDelay(long delay, TimeUnit unit) {
124                baseTime = System.currentTimeMillis() + unit.toMillis(delay);
125            }
126    
127            /**
128             * Return the delay of the element.
129             *
130             * @param unit time unit of the delay.
131             *
132             * @return the delay in the specified time unit.
133             */
134            public long getDelay(TimeUnit unit) {
135                return unit.convert(baseTime - System.currentTimeMillis(), TimeUnit.MILLISECONDS);
136            }
137    
138            /**
139             * Compare the age of this wrapper element with another. The priority is not used for the comparision.
140             *
141             * @param o the other wrapper element to compare with.
142             *
143             * @return less than zero if this wrapper is older, zero if both wrapper elements have the same age, greater
144             *         than zero if the parameter wrapper element is older.
145             */
146            public int compareTo(Delayed o) {
147                long diff = (getDelay(TimeUnit.MILLISECONDS) - o.getDelay(TimeUnit.MILLISECONDS));
148                if(diff > 0) {
149                    return 1;
150                } else if(diff < 0) {
151                    return -1;
152                } else {
153                    return 0;
154                }
155            }
156    
157            /**
158             * Return the string representation of the wrapper element.
159             *
160             * @return the string representation of the wrapper element.
161             */
162            @Override
163            public String toString() {
164                StringBuilder sb = new StringBuilder();
165                sb.append("[").append(element).append("] priority=").append(priority).append(" delay=").
166                        append(getDelay(TimeUnit.MILLISECONDS));
167                return sb.toString();
168            }
169    
170        }
171    
172        /**
173         * Frequency, in milliseconds, of the anti-starvation check.
174         */
175        public static final long ANTI_STARVATION_INTERVAL = 500;
176    
177        private int priorities;
178        private DelayQueue<QueueElement<E>>[] queues;
179        private transient final ReentrantLock lock = new ReentrantLock();
180        private transient long lastAntiStarvationCheck = 0;
181        private long maxWait;
182        private int maxSize;
183        private AtomicInteger currentSize;
184    
185        /**
186         * Create a <code>PriorityDelayQueue</code>.
187         *
188         * @param priorities number of priorities the queue will support.
189         * @param maxWait max wait time for elements before they are promoted to the next higher priority.
190         * @param unit time unit of the max wait time.
191         * @param maxSize maximum size of the queue, -1 means unbounded.
192         */
193        @SuppressWarnings("unchecked")
194        public PriorityDelayQueue(int priorities, long maxWait, TimeUnit unit, int maxSize) {
195            if (priorities < 1) {
196                throw new IllegalArgumentException("priorities must be 1 or more");
197            }
198            if (maxWait < 0) {
199                throw new IllegalArgumentException("maxWait must be greater than 0");
200            }
201            if (maxSize < -1 || maxSize == 0) {
202                throw new IllegalArgumentException("maxSize must be -1 or greater than 0");
203            }
204            this.priorities = priorities;
205            queues = new DelayQueue[priorities];
206            for (int i = 0; i < priorities; i++) {
207                queues[i] = new DelayQueue<QueueElement<E>>();
208            }
209            this.maxWait = unit.toMillis(maxWait);
210            this.maxSize = maxSize;
211            if (maxSize != -1) {
212                currentSize = new AtomicInteger();
213            }
214        }
215    
216        /**
217         * Return number of priorities the queue supports.
218         *
219         * @return number of priorities the queue supports.
220         */
221        public int getPriorities() {
222            return priorities;
223        }
224    
225        /**
226         * Return the max wait time for elements before they are promoted to the next higher priority.
227         *
228         * @param unit time unit of the max wait time.
229         *
230         * @return the max wait time in the specified time unit.
231         */
232        public long getMaxWait(TimeUnit unit) {
233            return unit.convert(maxWait, TimeUnit.MILLISECONDS);
234        }
235    
236        /**
237         * Return the maximum queue size.
238         *
239         * @return the maximum queue size. If <code>-1</code> the queue is unbounded.
240         */
241        public long getMaxSize() {
242            return maxSize;
243        }
244    
245        /**
246         * Return an iterator over all the {@link QueueElement} elements (both expired and unexpired) in this queue. The
247         * iterator does not return the elements in any particular order.  The returned <tt>Iterator</tt> is a "weakly
248         * consistent" iterator that will never throw {@link ConcurrentModificationException}, and guarantees to traverse
249         * elements as they existed upon construction of the iterator, and may (but is not guaranteed to) reflect any
250         * modifications subsequent to construction.
251         *
252         * @return an iterator over the {@link QueueElement} elements in this queue.
253         */
254        @Override
255        @SuppressWarnings("unchecked")
256        public Iterator<QueueElement<E>> iterator() {
257            QueueElement[][] queueElements = new QueueElement[queues.length][];
258            try {
259                lock.lock();
260                for (int i = 0; i < queues.length; i++) {
261                    queueElements[i] = queues[i].toArray(new QueueElement[0]);
262                }
263            }
264            finally {
265                lock.unlock();
266            }
267            List<QueueElement<E>> list = new ArrayList<QueueElement<E>>();
268            for (QueueElement[] elements : queueElements) {
269                list.addAll(Arrays.asList((QueueElement<E>[]) elements));
270            }
271            return list.iterator();
272        }
273    
274        /**
275         * Return the number of elements in the queue.
276         *
277         * @return the number of elements in the queue.
278         */
279        @Override
280        public int size() {
281            int size = 0;
282            for (DelayQueue<QueueElement<E>> queue : queues) {
283                size += queue.size();
284            }
285            return size;
286        }
287    
288        /**
289         * Return the number of elements on each priority sub-queue.
290         *
291         * @return the number of elements on each priority sub-queue.
292         */
293        public int[] sizes() {
294            int[] sizes = new int[queues.length];
295            for (int i = 0; i < queues.length; i++) {
296                sizes[i] = queues[i].size();
297            }
298            return sizes;
299        }
300    
301        /**
302         * Inserts the specified element into this queue if it is possible to do
303         * so immediately without violating capacity restrictions, returning
304         * <tt>true</tt> upon success and throwing an
305         * <tt>IllegalStateException</tt> if no space is currently available.
306         * When using a capacity-restricted queue, it is generally preferable to
307         * use {@link #offer(Object) offer}.
308         *
309         * @param queueElement the {@link QueueElement} element to add.
310         * @return <tt>true</tt> (as specified by {@link Collection#add})
311         * @throws IllegalStateException if the element cannot be added at this
312         *         time due to capacity restrictions
313         * @throws ClassCastException if the class of the specified element
314         *         prevents it from being added to this queue
315         * @throws NullPointerException if the specified element is null
316         * @throws IllegalArgumentException if some property of the specified
317         *         element prevents it from being added to this queue
318         */
319        @Override
320        public boolean add(QueueElement<E> queueElement) {
321            return offer(queueElement, false);
322        }
323    
324        /**
325         * Insert the specified {@link QueueElement} element into the queue.
326         *
327         * @param queueElement the {@link QueueElement} element to add.
328         * @param ignoreSize if the queue is bound to a maximum size and the maximum size is reached, this parameter (if set
329         * to <tt>true</tt>) allows to ignore the maximum size and add the element to the queue.
330         *
331         * @return <tt>true</tt> if the element has been inserted, <tt>false</tt> if the element was not inserted (the queue
332         *         has reached its maximum size).
333         *
334         * @throws NullPointerException if the specified element is null
335         */
336        boolean offer(QueueElement<E> queueElement, boolean ignoreSize) {
337            if (queueElement == null) {
338                throw new NullPointerException("queueElement is NULL");
339            }
340            if (queueElement.getPriority() < 0 && queueElement.getPriority() >= priorities) {
341                throw new IllegalArgumentException("priority out of range");
342            }
343            if (queueElement.inQueue) {
344                throw new IllegalStateException("queueElement already in a queue");
345            }
346            if (!ignoreSize && currentSize != null && currentSize.get() >= maxSize) {
347                return false;
348            }
349            boolean accepted = queues[queueElement.getPriority()].offer(queueElement);
350            debug("offer([{0}]), to P[{1}] delay[{2}ms] accepted[{3}]", queueElement.getElement().toString(),
351                  queueElement.getPriority(), queueElement.getDelay(TimeUnit.MILLISECONDS), accepted);
352            if (accepted) {
353                if (currentSize != null) {
354                    currentSize.incrementAndGet();
355                }
356                queueElement.inQueue = true;
357            }
358            return accepted;
359        }
360    
361        /**
362         * Insert the specified element into the queue.
363         * <p/>
364         * The element is added with minimun priority and no delay.
365         *
366         * @param queueElement the element to add.
367         *
368         * @return <tt>true</tt> if the element has been inserted, <tt>false</tt> if the element was not inserted (the queue
369         *         has reached its maximum size).
370         *
371         * @throws NullPointerException if the specified element is null
372         */
373        @Override
374        public boolean offer(QueueElement<E> queueElement) {
375            return offer(queueElement, false);
376        }
377    
378        /**
379         * Retrieve and remove the head of this queue, or return <tt>null</tt> if this queue has no elements with an expired
380         * delay.
381         * <p/>
382         * The retrieved element is the oldest one from the highest priority sub-queue.
383         * <p/>
384         * Invocations to this method run the anti-starvation (once every interval check).
385         *
386         * @return the head of this queue, or <tt>null</tt> if this queue has no elements with an expired delay.
387         */
388        @Override
389        public QueueElement<E> poll() {
390            try {
391                lock.lock();
392                antiStarvation();
393                QueueElement<E> e = null;
394                int i = priorities;
395                for (; e == null && i > 0; i--) {
396                    e = queues[i - 1].poll();
397                }
398                if (e != null) {
399                    if (currentSize != null) {
400                        currentSize.decrementAndGet();
401                    }
402                    e.inQueue = false;
403                    debug("poll(): [{1}], from P[{2}]", e.getElement().toString(), i);
404                }
405                return e;
406            }
407            finally {
408                lock.unlock();
409            }
410        }
411    
412        /**
413         * Retrieve, but does not remove, the head of this queue, or returns <tt>null</tt> if this queue is empty.  Unlike
414         * <tt>poll</tt>, if no expired elements are available in the queue, this method returns the element that will
415         * expire next, if one exists.
416         *
417         * @return the head of this queue, or <tt>null</tt> if this queue is empty.
418         */
419        @Override
420        public QueueElement<E> peek() {
421            try {
422                lock.lock();
423                antiStarvation();
424                QueueElement<E> e = null;
425    
426                QueueElement<E> [] seeks = new QueueElement[priorities];
427                boolean foundElement = false;
428                for (int i = priorities - 1; i > -1; i--) {
429                    e = queues[i].peek();
430                    debug("peek(): considering [{0}] from P[{1}]", e, i);
431                    seeks[priorities - i - 1] = e;
432                    foundElement |= e != null;
433                }
434                if (foundElement) {
435                    e = null;
436                    for (int i = 0; e == null && i < priorities; i++) {
437                        if (seeks[i] != null && seeks[i].getDelay(TimeUnit.MILLISECONDS) > 0) {
438                            debug("peek, ignoring [{0}]", seeks[i]);
439                        }
440                        else {
441                            e = seeks[i];
442                        }
443                    }
444                    if (e != null) {
445                        debug("peek(): choosing [{0}]", e);
446                    }
447                    if (e == null) {
448                        int first;
449                        for (first = 0; e == null && first < priorities; first++) {
450                            e = seeks[first];
451                        }
452                        if (e != null) {
453                            debug("peek(): initial choosing [{0}]", e);
454                        }
455                        for (int i = first; i < priorities; i++) {
456                            QueueElement<E> ee = seeks[i];
457                            if (ee != null && ee.getDelay(TimeUnit.MILLISECONDS) < e.getDelay(TimeUnit.MILLISECONDS)) {
458                                debug("peek(): choosing [{0}] over [{1}]", ee, e);
459                                e = ee;
460                            }
461                        }
462                    }
463                }
464                if (e != null) {
465                    debug("peek(): [{0}], from P[{1}]", e.getElement().toString(), e.getPriority());
466                }
467                else {
468                    debug("peek(): NULL");
469                }
470                return e;
471            }
472            finally {
473                lock.unlock();
474            }
475        }
476    
477        /**
478         * Run the anti-starvation check every {@link #ANTI_STARVATION_INTERVAL} milliseconds.
479         * <p/>
480         * It promotes elements beyond max wait time to the next higher priority sub-queue.
481         */
482        private void antiStarvation() {
483            long now = System.currentTimeMillis();
484            if (now - lastAntiStarvationCheck > ANTI_STARVATION_INTERVAL) {
485                for (int i = 0; i < queues.length - 1; i++) {
486                    antiStarvation(queues[i], queues[i + 1], "from P[" + i + "] to P[" + (i + 1) + "]");
487                }
488                StringBuilder sb = new StringBuilder();
489                for (int i = 0; i < queues.length; i++) {
490                    sb.append("P[").append(i).append("]=").append(queues[i].size()).append(" ");
491                }
492                debug("sub-queue sizes: {0}", sb.toString());
493                lastAntiStarvationCheck = System.currentTimeMillis();
494            }
495        }
496    
497        /**
498         * Promote elements beyond max wait time from a lower priority sub-queue to a higher priority sub-queue.
499         *
500         * @param lowerQ lower priority sub-queue.
501         * @param higherQ higher priority sub-queue.
502         * @param msg sub-queues msg (from-to) for debugging purposes.
503         */
504        private void antiStarvation(DelayQueue<QueueElement<E>> lowerQ, DelayQueue<QueueElement<E>> higherQ, String msg) {
505            int moved = 0;
506            QueueElement<E> e = lowerQ.poll();
507            while (e != null && e.getDelay(TimeUnit.MILLISECONDS) < -maxWait) {
508                e.setDelay(0, TimeUnit.MILLISECONDS);
509                if (!higherQ.offer(e)) {
510                    throw new IllegalStateException("Could not move element to higher sub-queue, element rejected");
511                }
512                e.priority++;
513                e = lowerQ.poll();
514                moved++;
515            }
516            if (e != null) {
517                if (!lowerQ.offer(e)) {
518                    throw new IllegalStateException("Could not reinsert element to current sub-queue, element rejected");
519                }
520            }
521            debug("anti-starvation, moved {0} element(s) {1}", moved, msg);
522        }
523    
524        /**
525         * Method for debugging purposes. This implementation is a <tt>NOP</tt>.
526         * <p/>
527         * This method should be overriden for logging purposes.
528         * <p/>
529         * Message templates used by this class are in JDK's <tt>MessageFormat</tt> syntax.
530         *
531         * @param msgTemplate message template.
532         * @param msgArgs arguments for the message template.
533         */
534        protected void debug(String msgTemplate, Object... msgArgs) {
535        }
536    
537        //BlockingQueue implementation
538    
539        /**
540         * Insert the specified element into this queue, waiting if necessary
541         * for space to become available.
542         * <p/>
543         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
544         *
545         * @param e the element to add
546         * @throws InterruptedException if interrupted while waiting
547         * @throws ClassCastException if the class of the specified element
548         *         prevents it from being added to this queue
549         * @throws NullPointerException if the specified element is null
550         * @throws IllegalArgumentException if some property of the specified
551         *         element prevents it from being added to this queue
552         */
553        @Override
554        public void put(QueueElement<E> e) throws InterruptedException {
555            while (!offer(e, true)) {
556                Thread.sleep(10);
557            }
558        }
559    
560        /**
561         * Insert the specified element into this queue, waiting up to the
562         * specified wait time if necessary for space to become available.
563         * <p/>
564         * IMPORTANT: This implementation forces the addition of the element to the queue regardless
565         * of the queue current size. The timeout value is ignored as the element is added immediately.
566         * <p/>
567         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
568         *
569         * @param e the element to add
570         * @param timeout how long to wait before giving up, in units of
571         *        <tt>unit</tt>
572         * @param unit a <tt>TimeUnit</tt> determining how to interpret the
573         *        <tt>timeout</tt> parameter
574         * @return <tt>true</tt> if successful, or <tt>false</tt> if
575         *         the specified waiting time elapses before space is available
576         * @throws InterruptedException if interrupted while waiting
577         * @throws ClassCastException if the class of the specified element
578         *         prevents it from being added to this queue
579         * @throws NullPointerException if the specified element is null
580         * @throws IllegalArgumentException if some property of the specified
581         *         element prevents it from being added to this queue
582         */
583        @Override
584        public boolean offer(QueueElement<E> e, long timeout, TimeUnit unit) throws InterruptedException {
585            return offer(e, true);
586        }
587    
588        /**
589         * Retrieve and removes the head of this queue, waiting if necessary
590         * until an element becomes available.
591         * <p/>
592         * IMPORTANT: This implementation has a delay of up to 10ms (when the queue is empty) to detect a new element
593         * is available. It is doing a 10ms sleep.
594         * <p/>
595         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
596         *
597         * @return the head of this queue
598         * @throws InterruptedException if interrupted while waiting
599         */
600        @Override
601        public QueueElement<E> take() throws InterruptedException {
602            QueueElement<E> e = poll();
603            while (e == null) {
604                Thread.sleep(10);
605                e = poll();
606            }
607            return e;
608        }
609    
610        /**
611         * Retrieve and removes the head of this queue, waiting up to the
612         * specified wait time if necessary for an element to become available.
613         * <p/>
614         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
615         *
616         * @param timeout how long to wait before giving up, in units of
617         *        <tt>unit</tt>
618         * @param unit a <tt>TimeUnit</tt> determining how to interpret the
619         *        <tt>timeout</tt> parameter
620         * @return the head of this queue, or <tt>null</tt> if the
621         *         specified waiting time elapses before an element is available
622         * @throws InterruptedException if interrupted while waiting
623         */
624        @Override
625        public QueueElement<E> poll(long timeout, TimeUnit unit) throws InterruptedException {
626            QueueElement<E> e = poll();
627            long time = System.currentTimeMillis() + unit.toMillis(timeout);
628            while (e == null && time > System.currentTimeMillis()) {
629                Thread.sleep(10);
630                e = poll();
631            }
632            return poll();
633        }
634    
635        /**
636         * Return the number of additional elements that this queue can ideally
637         * (in the absence of memory or resource constraints) accept without
638         * blocking, or <tt>Integer.MAX_VALUE</tt> if there is no intrinsic
639         * limit.
640         *
641         * <p>Note that you <em>cannot</em> always tell if an attempt to insert
642         * an element will succeed by inspecting <tt>remainingCapacity</tt>
643         * because it may be the case that another thread is about to
644         * insert or remove an element.
645         * <p/>
646         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
647         *
648         * @return the remaining capacity
649         */
650        @Override
651        public int remainingCapacity() {
652            return (maxSize == -1) ? -1 : maxSize - size();
653        }
654    
655        /**
656         * Remove all available elements from this queue and adds them
657         * to the given collection.  This operation may be more
658         * efficient than repeatedly polling this queue.  A failure
659         * encountered while attempting to add elements to
660         * collection <tt>c</tt> may result in elements being in neither,
661         * either or both collections when the associated exception is
662         * thrown.  Attempt to drain a queue to itself result in
663         * <tt>IllegalArgumentException</tt>. Further, the behavior of
664         * this operation is undefined if the specified collection is
665         * modified while the operation is in progress.
666         * <p/>
667         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
668         *
669         * @param c the collection to transfer elements into
670         * @return the number of elements transferred
671         * @throws UnsupportedOperationException if addition of elements
672         *         is not supported by the specified collection
673         * @throws ClassCastException if the class of an element of this queue
674         *         prevents it from being added to the specified collection
675         * @throws NullPointerException if the specified collection is null
676         * @throws IllegalArgumentException if the specified collection is this
677         *         queue, or some property of an element of this queue prevents
678         *         it from being added to the specified collection
679         */
680        @Override
681        public int drainTo(Collection<? super QueueElement<E>> c) {
682            int count = 0;
683            for (DelayQueue<QueueElement<E>> q : queues) {
684                count += q.drainTo(c);
685            }
686            return count;
687        }
688    
689        /**
690         * Remove at most the given number of available elements from
691         * this queue and adds them to the given collection.  A failure
692         * encountered while attempting to add elements to
693         * collection <tt>c</tt> may result in elements being in neither,
694         * either or both collections when the associated exception is
695         * thrown.  Attempt to drain a queue to itself result in
696         * <tt>IllegalArgumentException</tt>. Further, the behavior of
697         * this operation is undefined if the specified collection is
698         * modified while the operation is in progress.
699         * <p/>
700         * NOTE: This method is to fulfill the <tt>BlockingQueue<tt/> interface. Not implemented in the most optimal way.
701         *
702         * @param c the collection to transfer elements into
703         * @param maxElements the maximum number of elements to transfer
704         * @return the number of elements transferred
705         * @throws UnsupportedOperationException if addition of elements
706         *         is not supported by the specified collection
707         * @throws ClassCastException if the class of an element of this queue
708         *         prevents it from being added to the specified collection
709         * @throws NullPointerException if the specified collection is null
710         * @throws IllegalArgumentException if the specified collection is this
711         *         queue, or some property of an element of this queue prevents
712         *         it from being added to the specified collection
713         */
714        @Override
715        public int drainTo(Collection<? super QueueElement<E>> c, int maxElements) {
716            int left = maxElements;
717            int count = 0;
718            for (DelayQueue<QueueElement<E>> q : queues) {
719                int drained = q.drainTo(c, left);
720                count += drained;
721                left -= drained;
722            }
723            return count;
724        }
725    
726        /**
727         * Removes all of the elements from this queue. The queue will be empty after this call returns.
728         */
729        @Override
730        public void clear() {
731            for (DelayQueue<QueueElement<E>> q : queues) {
732                q.clear();
733            }
734        }
735    }