From 10574bce818ddf316b4d89ca2d3fc91ab1160b7b Mon Sep 17 00:00:00 2001 From: Konstantin V Shvachko Date: Thu, 16 Jun 2016 19:09:21 -0700 Subject: [PATCH] HADOOP-13189. FairCallQueue makes callQueue larger than the configured capacity. Contributed by Vinitha Gankidi. --- .../apache/hadoop/ipc/CallQueueManager.java | 3 ++- .../org/apache/hadoop/ipc/FairCallQueue.java | 19 +++++++++++++------ .../apache/hadoop/ipc/TestFairCallQueue.java | 8 +------- 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java index 0b1de936121..08274c3a42a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java @@ -51,7 +51,8 @@ public class CallQueueManager { maxQueueSize, namespace, conf); this.putRef = new AtomicReference>(bq); this.takeRef = new AtomicReference>(bq); - LOG.info("Using callQueue " + backingClass); + LOG.info("Using callQueue: " + backingClass + " queueCapacity: " + + maxQueueSize); } private > T createCallQueueInstance( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java index 0b56243db58..6a6701aaf5f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/FairCallQueue.java @@ -77,21 +77,28 @@ public class FairCallQueue extends AbstractQueue /** * Create a FairCallQueue. - * @param capacity the maximum size of each sub-queue + * @param capacity the total size of all sub-queues * @param ns the prefix to use for configuration * @param conf the configuration to read from - * Notes: the FairCallQueue has no fixed capacity. Rather, it has a minimum - * capacity of `capacity` and a maximum capacity of `capacity * number_queues` + * Notes: Each sub-queue has a capacity of `capacity / numSubqueues`. + * The first or the highest priority sub-queue has an excess capacity + * of `capacity % numSubqueues` */ public FairCallQueue(int capacity, String ns, Configuration conf) { int numQueues = parseNumQueues(ns, conf); - LOG.info("FairCallQueue is in use with " + numQueues + " queues."); + LOG.info("FairCallQueue is in use with " + numQueues + + " queues with total capacity of " + capacity); this.queues = new ArrayList>(numQueues); this.overflowedCalls = new ArrayList(numQueues); - + int queueCapacity = capacity / numQueues; + int capacityForFirstQueue = queueCapacity + (capacity % numQueues); for(int i=0; i < numQueues; i++) { - this.queues.add(new LinkedBlockingQueue(capacity)); + if (i == 0) { + this.queues.add(new LinkedBlockingQueue(capacityForFirstQueue)); + } else { + this.queues.add(new LinkedBlockingQueue(queueCapacity)); + } this.overflowedCalls.add(new AtomicLong(0)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java index 2694ba3ab91..bc1b5885633 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestFairCallQueue.java @@ -18,12 +18,6 @@ package org.apache.hadoop.ipc; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -64,7 +58,7 @@ public class TestFairCallQueue extends TestCase { Configuration conf = new Configuration(); conf.setInt("ns." + IPC_CALLQUEUE_PRIORITY_LEVELS_KEY, 2); - fcq = new FairCallQueue(5, "ns", conf); + fcq = new FairCallQueue(10, "ns", conf); } //