You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2016/06/09 03:22:59 UTC

hbase git commit: Revert "HBASE-15994 Allow selection of RpcSchedulers Adds logging by the RpcExecutors of their run configs Changes the default RpcSchedulerFactory from SimpleRpcSchedulerFactory.class to RpcSchedulerFactoryImpl.class. RpcSchedulerFactor

Repository: hbase
Updated Branches:
  refs/heads/master 3ac4a57fd -> 407aa4d49


Revert "HBASE-15994 Allow selection of RpcSchedulers Adds logging by the RpcExecutors of their run configs Changes the default RpcSchedulerFactory from SimpleRpcSchedulerFactory.class to RpcSchedulerFactoryImpl.class. RpcSchedulerFactoryImpl.class. defaults to using SimpleRpcSchedulerFactory.class and the SimpleRpcScheduler, as has been default up to this, unless you set "hbase.region.server.rpc.scheduler.class" to org.apache.hadoop.hbase.ipc.FifoRpcScheduler"

This reverts commit 3ac4a57fd205e1909c874cabd7fda9fb176f3f0f.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/407aa4d4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/407aa4d4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/407aa4d4

Branch: refs/heads/master
Commit: 407aa4d4963eb0d4aa3e0950babc8cec31367e11
Parents: 3ac4a57
Author: stack <st...@apache.org>
Authored: Wed Jun 8 20:22:50 2016 -0700
Committer: stack <st...@apache.org>
Committed: Wed Jun 8 20:22:50 2016 -0700

----------------------------------------------------------------------
 .../hbase/ipc/BalancedQueueRpcExecutor.java     |  4 --
 .../hadoop/hbase/ipc/FifoRpcScheduler.java      |  5 --
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    | 54 +++++++--------
 .../regionserver/FifoRpcSchedulerFactory.java   | 47 -------------
 .../hbase/regionserver/RpcSchedulerFactory.java |  4 +-
 .../regionserver/SimpleRpcSchedulerFactory.java |  6 +-
 .../regionserver/TestRpcSchedulerFactory.java   | 71 --------------------
 7 files changed, 30 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
index 3505221..e4205eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
@@ -22,8 +22,6 @@ import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -38,7 +36,6 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
 @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX })
 @InterfaceStability.Evolving
 public class BalancedQueueRpcExecutor extends RpcExecutor {
-  private static final Log LOG = LogFactory.getLog(BalancedQueueRpcExecutor.class);
 
   protected final List<BlockingQueue<CallRunner>> queues;
   private final QueueBalancer balancer;
@@ -65,7 +62,6 @@ public class BalancedQueueRpcExecutor extends RpcExecutor {
     queues = new ArrayList<BlockingQueue<CallRunner>>(numQueues);
     this.balancer = getBalancer(numQueues);
     initializeQueues(numQueues, queueClass, initargs);
-    LOG.debug(name + " queues=" + numQueues + " handlerCount=" + handlerCount);
   }
 
   protected void initializeQueues(final int numQueues,

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
index 70d903a..ee36f3f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DaemonThreadFactory;
 
@@ -34,7 +32,6 @@ import java.util.concurrent.atomic.AtomicInteger;
  * This can be used for HMaster, where no prioritization is needed.
  */
 public class FifoRpcScheduler extends RpcScheduler {
-  private static final Log LOG = LogFactory.getLog(FifoRpcScheduler.class);
   private final int handlerCount;
   private final int maxQueueLength;
   private final AtomicInteger queueSize = new AtomicInteger(0);
@@ -44,8 +41,6 @@ public class FifoRpcScheduler extends RpcScheduler {
     this.handlerCount = handlerCount;
     this.maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
         handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
-    LOG.info("Using " + this.getClass().getSimpleName() + " as user call queue; handlerCount=" +
-        handlerCount + "; maxQueueLength=" + maxQueueLength);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index d9d61c1..431aeeb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -34,11 +34,8 @@ import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
 
 /**
- * The default scheduler. Configurable. Maintains isolated handler pools for general ('default'),
- * high-priority ('priority'), and replication ('replication') requests. Default behavior is to
- * balance the requests across handlers. Add configs to enable balancing by read vs writes, etc.
- * See below article for explanation of options.
- * @see <a href="http://blog.cloudera.com/blog/2014/12/new-in-cdh-5-2-improvements-for-running-multiple-workloads-on-a-single-hbase-cluster/">Overview on Request Queuing</a>
+ * A scheduler that maintains isolated handler pools for general,
+ * high-priority, and replication requests.
  */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
@@ -52,8 +49,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY =
       "hbase.ipc.server.callqueue.handler.factor";
 
-  /** If set to 'deadline', the default, uses a priority queue and deprioritizes long-running scans
-   */
+  /** If set to 'deadline', uses a priority queue and deprioritize long-running scans */
   public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
   public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
   public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
@@ -194,58 +190,54 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
 
     float callQueuesHandlersFactor = conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0);
     int numCallQueues = Math.max(1, (int)Math.round(handlerCount * callQueuesHandlersFactor));
-    LOG.info("Using " + callQueueType + " as user call queue; numCallQueues=" + numCallQueues +
-        "; callQReadShare=" + callqReadShare + ", callQScanShare=" + callqScanShare);
+
+    LOG.info("Using " + callQueueType + " as user call queue, count=" + numCallQueues);
+
     if (numCallQueues > 1 && callqReadShare > 0) {
       // multiple read/write queues
-      if (isDeadlineQueueType(callQueueType)) {
+      if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
+        callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
             callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
             BoundedPriorityBlockingQueue.class, callPriority);
       } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
         Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
           codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount,
+        callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount,
           numCallQueues, callqReadShare, callqScanShare,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
           AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
       } else {
-        callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
+        callExecutor = new RWQueueRpcExecutor("RW.default", handlerCount, numCallQueues,
           callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
       }
     } else {
       // multiple queues
-      if (isDeadlineQueueType(callQueueType)) {
+      if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
         CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
-        callExecutor =
-          new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
-            conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
+        callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
+          conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
       } else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
-        callExecutor =
-          new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
-            conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
-            codelTargetDelay, codelInterval, codelLifoThreshold,
-            numGeneralCallsDropped, numLifoModeSwitches);
+        callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount, numCallQueues,
+          conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
+          codelTargetDelay, codelInterval, codelLifoThreshold,
+          numGeneralCallsDropped, numLifoModeSwitches);
       } else {
-        callExecutor = new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount,
+        callExecutor = new BalancedQueueRpcExecutor("B.default", handlerCount,
             numCallQueues, maxQueueLength, conf, abortable);
       }
     }
+
     // Create 2 queues to help priorityExecutor be more scalable.
     this.priorityExecutor = priorityHandlerCount > 0 ?
-      new BalancedQueueRpcExecutor("BalancedQ.priority", priorityHandlerCount, 2,
-          maxPriorityQueueLength):
-      null;
+        new BalancedQueueRpcExecutor("Priority", priorityHandlerCount, 2, maxPriorityQueueLength) :
+        null;
+
    this.replicationExecutor =
-     replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("BalancedQ.replication",
+     replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("Replication",
        replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
   }
 
-  private static boolean isDeadlineQueueType(final String callQueueType) {
-    return callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
-  }
-
   public SimpleRpcScheduler(
 	      Configuration conf,
 	      int handlerCount,

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FifoRpcSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FifoRpcSchedulerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FifoRpcSchedulerFactory.java
deleted file mode 100644
index f4b51ba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FifoRpcSchedulerFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.regionserver;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
-import org.apache.hadoop.hbase.ipc.PriorityFunction;
-import org.apache.hadoop.hbase.ipc.RpcScheduler;
-
-/**
- * Factory to use when you want to use the {@link FifoRpcScheduler}
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class FifoRpcSchedulerFactory implements RpcSchedulerFactory {
-  @Override
-  public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) {
-    int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
-      HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);
-    return new FifoRpcScheduler(conf, handlerCount);
-  }
-
-  @Deprecated
-  @Override
-  public RpcScheduler create(Configuration conf, PriorityFunction priority) {
-    return create(conf, priority, null);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RpcSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RpcSchedulerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RpcSchedulerFactory.java
index 7bc59da..f554781 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RpcSchedulerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RpcSchedulerFactory.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ipc.RpcScheduler;
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
 public interface RpcSchedulerFactory {
+
   /**
    * Constructs a {@link org.apache.hadoop.hbase.ipc.RpcScheduler}.
    */
@@ -38,4 +39,5 @@ public interface RpcSchedulerFactory {
 
   @Deprecated
   RpcScheduler create(Configuration conf, PriorityFunction priority);
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
index 92462c8..743c5bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.ipc.PriorityFunction;
 import org.apache.hadoop.hbase.ipc.RpcScheduler;
 import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
 
-/** Constructs a {@link SimpleRpcScheduler}.
- */
+/** Constructs a {@link SimpleRpcScheduler}. */
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
 public class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
+
   @Override
   @Deprecated
   public RpcScheduler create(Configuration conf, PriorityFunction priority) {
@@ -42,6 +42,7 @@ public class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
   public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) {
     int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
         HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);
+
     return new SimpleRpcScheduler(
       conf,
       handlerCount,
@@ -53,4 +54,5 @@ public class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
       server,
       HConstants.QOS_THRESHOLD);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/407aa4d4/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java
deleted file mode 100644
index 9366c54..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * 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.regionserver;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
-import org.apache.hadoop.hbase.ipc.RpcScheduler;
-import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.junit.rules.TestRule;
-
-/**
- * A silly test that does nothing but make sure an rpcscheduler factory makes what it says
- * it is going to make.
- */
-@Category(SmallTests.class)
-public class TestRpcSchedulerFactory {
-  @Rule public TestName testName = new TestName();
-  @ClassRule public static TestRule timeout =
-      CategoryBasedTimeout.forClass(TestRpcSchedulerFactory.class);
-  private Configuration conf;
-
-  @Before
-  public void setUp() throws Exception {
-    this.conf = HBaseConfiguration.create();
-  }
-
-  @Test
-  public void testRWQ() {
-    // Set some configs just to see how it changes the scheduler. Can't assert the settings had
-    // an effect. Just eyeball the log.
-    this.conf.setDouble(SimpleRpcScheduler.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5);
-    this.conf.setDouble(SimpleRpcScheduler.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0.5);
-    this.conf.setDouble(SimpleRpcScheduler.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0.5);
-    RpcSchedulerFactory factory = new SimpleRpcSchedulerFactory();
-    RpcScheduler rpcScheduler = factory.create(this.conf, null, null);
-    assertTrue(rpcScheduler.getClass().equals(SimpleRpcScheduler.class));
-  }
-
-  @Test
-  public void testFifo() {
-    RpcSchedulerFactory factory = new FifoRpcSchedulerFactory();
-    RpcScheduler rpcScheduler = factory.create(this.conf, null, null);
-    assertTrue(rpcScheduler.getClass().equals(FifoRpcScheduler.class));
-  }
-}
\ No newline at end of file