HADOOP-14033. Reduce fair call queue lock contention. Contributed by Daryn Sharp.
(cherry picked from commit 0c01cf5798
)
This commit is contained in:
parent
41361ec39e
commit
5de6f1b791
|
@ -27,8 +27,7 @@ import java.util.AbstractQueue;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.Semaphore;
|
||||||
import java.util.concurrent.locks.Condition;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
@ -55,16 +54,15 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
/* The queues */
|
/* The queues */
|
||||||
private final ArrayList<BlockingQueue<E>> queues;
|
private final ArrayList<BlockingQueue<E>> queues;
|
||||||
|
|
||||||
/* Read locks */
|
/* Track available permits for scheduled objects. All methods that will
|
||||||
private final ReentrantLock takeLock = new ReentrantLock();
|
* mutate a subqueue must acquire or release a permit on the semaphore.
|
||||||
private final Condition notEmpty = takeLock.newCondition();
|
* A semaphore is much faster than an exclusive lock because producers do
|
||||||
|
* not contend with consumers and consumers do not block other consumers
|
||||||
|
* while polling.
|
||||||
|
*/
|
||||||
|
private final Semaphore semaphore = new Semaphore(0);
|
||||||
private void signalNotEmpty() {
|
private void signalNotEmpty() {
|
||||||
takeLock.lock();
|
semaphore.release();
|
||||||
try {
|
|
||||||
notEmpty.signal();
|
|
||||||
} finally {
|
|
||||||
takeLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/* Multiplexer picks which queue to draw from */
|
/* Multiplexer picks which queue to draw from */
|
||||||
|
@ -112,28 +110,25 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the first non-empty queue with equal to <i>startIdx</i>, or
|
* Returns an element first non-empty queue equal to the priority returned
|
||||||
* or scans from highest to lowest priority queue.
|
* by the multiplexer or scans from highest to lowest priority queue.
|
||||||
|
*
|
||||||
|
* Caller must always acquire a semaphore permit before invoking.
|
||||||
*
|
*
|
||||||
* @param startIdx the queue number to start searching at
|
|
||||||
* @return the first non-empty queue with less priority, or null if
|
* @return the first non-empty queue with less priority, or null if
|
||||||
* everything was empty
|
* everything was empty
|
||||||
*/
|
*/
|
||||||
private BlockingQueue<E> getFirstNonEmptyQueue(int startIdx) {
|
private E removeNextElement() {
|
||||||
BlockingQueue<E> queue = this.queues.get(startIdx);
|
int priority = multiplexer.getAndAdvanceCurrentIndex();
|
||||||
if (queue.size() != 0) {
|
E e = queues.get(priority).poll();
|
||||||
return queue;
|
if (e == null) {
|
||||||
}
|
for (int idx = 0; e == null && idx < queues.size(); idx++) {
|
||||||
final int numQueues = this.queues.size();
|
e = queues.get(idx).poll();
|
||||||
for(int i=0; i < numQueues; i++) {
|
|
||||||
queue = this.queues.get(i);
|
|
||||||
if (queue.size() != 0) {
|
|
||||||
return queue;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// guaranteed to find an element if caller acquired permit.
|
||||||
// All queues were empty
|
assert e != null : "consumer didn't acquire semaphore!";
|
||||||
return null;
|
return e;
|
||||||
}
|
}
|
||||||
|
|
||||||
/* AbstractQueue and BlockingQueue methods */
|
/* AbstractQueue and BlockingQueue methods */
|
||||||
|
@ -184,9 +179,9 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
int priorityLevel = e.getPriorityLevel();
|
int priorityLevel = e.getPriorityLevel();
|
||||||
BlockingQueue<E> q = this.queues.get(priorityLevel);
|
BlockingQueue<E> q = this.queues.get(priorityLevel);
|
||||||
boolean ret = q.offer(e, timeout, unit);
|
boolean ret = q.offer(e, timeout, unit);
|
||||||
|
if (ret) {
|
||||||
signalNotEmpty();
|
signalNotEmpty();
|
||||||
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -195,72 +190,21 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
int priorityLevel = e.getPriorityLevel();
|
int priorityLevel = e.getPriorityLevel();
|
||||||
BlockingQueue<E> q = this.queues.get(priorityLevel);
|
BlockingQueue<E> q = this.queues.get(priorityLevel);
|
||||||
boolean ret = q.offer(e);
|
boolean ret = q.offer(e);
|
||||||
|
if (ret) {
|
||||||
signalNotEmpty();
|
signalNotEmpty();
|
||||||
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public E take() throws InterruptedException {
|
public E take() throws InterruptedException {
|
||||||
int startIdx = this.multiplexer.getAndAdvanceCurrentIndex();
|
semaphore.acquire();
|
||||||
|
return removeNextElement();
|
||||||
takeLock.lockInterruptibly();
|
|
||||||
try {
|
|
||||||
// Wait while queue is empty
|
|
||||||
for (;;) {
|
|
||||||
BlockingQueue<E> q = this.getFirstNonEmptyQueue(startIdx);
|
|
||||||
if (q != null) {
|
|
||||||
// Got queue, so return if we can poll out an object
|
|
||||||
E e = q.poll();
|
|
||||||
if (e != null) {
|
|
||||||
return e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
notEmpty.await();
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
takeLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public E poll(long timeout, TimeUnit unit)
|
public E poll(long timeout, TimeUnit unit) throws InterruptedException {
|
||||||
throws InterruptedException {
|
return semaphore.tryAcquire(timeout, unit) ? removeNextElement() : null;
|
||||||
|
|
||||||
int startIdx = this.multiplexer.getAndAdvanceCurrentIndex();
|
|
||||||
|
|
||||||
long nanos = unit.toNanos(timeout);
|
|
||||||
takeLock.lockInterruptibly();
|
|
||||||
try {
|
|
||||||
for (;;) {
|
|
||||||
BlockingQueue<E> q = this.getFirstNonEmptyQueue(startIdx);
|
|
||||||
if (q != null) {
|
|
||||||
E e = q.poll();
|
|
||||||
if (e != null) {
|
|
||||||
// Escape condition: there might be something available
|
|
||||||
return e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (nanos <= 0) {
|
|
||||||
// Wait has elapsed
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
// Now wait on the condition for a bit. If we get
|
|
||||||
// spuriously awoken we'll re-loop
|
|
||||||
nanos = notEmpty.awaitNanos(nanos);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
notEmpty.signal(); // propagate to a non-interrupted thread
|
|
||||||
throw ie;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
takeLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -269,15 +213,7 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public E poll() {
|
public E poll() {
|
||||||
int startIdx = this.multiplexer.getAndAdvanceCurrentIndex();
|
return semaphore.tryAcquire() ? removeNextElement() : null;
|
||||||
|
|
||||||
BlockingQueue<E> q = this.getFirstNonEmptyQueue(startIdx);
|
|
||||||
if (q == null) {
|
|
||||||
return null; // everything is empty
|
|
||||||
}
|
|
||||||
|
|
||||||
// Delegate to the sub-queue's poll, which could still return null
|
|
||||||
return q.poll();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -285,12 +221,11 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public E peek() {
|
public E peek() {
|
||||||
BlockingQueue<E> q = this.getFirstNonEmptyQueue(0);
|
E e = null;
|
||||||
if (q == null) {
|
for (int i=0; e == null && i < queues.size(); i++) {
|
||||||
return null;
|
e = queues.get(i).peek();
|
||||||
} else {
|
|
||||||
return q.peek();
|
|
||||||
}
|
}
|
||||||
|
return e;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -301,11 +236,7 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public int size() {
|
public int size() {
|
||||||
int size = 0;
|
return semaphore.availablePermits();
|
||||||
for (BlockingQueue<E> q : this.queues) {
|
|
||||||
size += q.size();
|
|
||||||
}
|
|
||||||
return size;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -324,20 +255,24 @@ public class FairCallQueue<E extends Schedulable> extends AbstractQueue<E>
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public int drainTo(Collection<? super E> c, int maxElements) {
|
public int drainTo(Collection<? super E> c, int maxElements) {
|
||||||
int sum = 0;
|
// initially take all permits to stop consumers from modifying queues
|
||||||
for (BlockingQueue<E> q : this.queues) {
|
// while draining. will restore any excess when done draining.
|
||||||
sum += q.drainTo(c, maxElements);
|
final int permits = semaphore.drainPermits();
|
||||||
|
final int numElements = Math.min(maxElements, permits);
|
||||||
|
int numRemaining = numElements;
|
||||||
|
for (int i=0; numRemaining > 0 && i < queues.size(); i++) {
|
||||||
|
numRemaining -= queues.get(i).drainTo(c, numRemaining);
|
||||||
}
|
}
|
||||||
return sum;
|
int drained = numElements - numRemaining;
|
||||||
|
if (permits > drained) { // restore unused permits.
|
||||||
|
semaphore.release(permits - drained);
|
||||||
|
}
|
||||||
|
return drained;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int drainTo(Collection<? super E> c) {
|
public int drainTo(Collection<? super E> c) {
|
||||||
int sum = 0;
|
return drainTo(c, Integer.MAX_VALUE);
|
||||||
for (BlockingQueue<E> q : this.queues) {
|
|
||||||
sum += q.drainTo(c);
|
|
||||||
}
|
|
||||||
return sum;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
Loading…
Reference in New Issue