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 2013/08/30 18:29:23 UTC

svn commit: r1519018 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/ipc/RpcServer.java main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java

Author: stack
Date: Fri Aug 30 16:29:23 2013
New Revision: 1519018

URL: http://svn.apache.org/r1519018
Log:
HBASE-9382 replicateWALEntry doesn't use the replication handlers

Added:
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java?rev=1519018&r1=1519017&r2=1519018&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java Fri Aug 30 16:29:23 2013
@@ -1719,10 +1719,10 @@ public class RpcServer implements RpcSer
       callQueueSize.add(totalRequestSize);
       Pair<RequestHeader, Message> headerAndParam =
         new Pair<RequestHeader, Message>(header, param);
-      if (priorityCallQueue != null && getQosLevel(headerAndParam) > highPriorityLevel) {
+      int level = getQosLevel(headerAndParam);
+      if (priorityCallQueue != null && level > highPriorityLevel) {
         priorityCallQueue.put(call);
-      } else if (replicationQueue != null &&
-          getQosLevel(headerAndParam) == HConstants.REPLICATION_QOS) {
+      } else if (replicationQueue != null && level == HConstants.REPLICATION_QOS) {
         replicationQueue.put(call);
       } else {
         callQueue.put(call);              // queue the call; maybe blocked here

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java?rev=1519018&r1=1519017&r2=1519018&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java Fri Aug 30 16:29:23 2013
@@ -99,7 +99,13 @@ class QosFunction implements Function<Pa
     for (Method m : HRegionServer.class.getMethods()) {
       QosPriority p = m.getAnnotation(QosPriority.class);
       if (p != null) {
-        qosMap.put(m.getName(), p.priority());
+        // Since we protobuf'd, and then subsequently, when we went with pb style, method names
+        // are capitalized.  This meant that this brittle compare of method names gotten by
+        // reflection no longer matched the method names comeing in over pb.  TODO: Get rid of this
+        // check.  For now, workaround is to capitalize the names we got from reflection so they
+        // have chance of matching the pb ones.
+        String capitalizedMethodName = capitalize(m.getName());
+        qosMap.put(capitalizedMethodName, p.priority());
       }
     }
     this.annotatedQos = qosMap;
@@ -119,6 +125,12 @@ class QosFunction implements Function<Pa
     }
   }
 
+  private String capitalize(final String s) {
+    StringBuilder strBuilder = new StringBuilder(s);
+    strBuilder.setCharAt(0, Character.toUpperCase(strBuilder.charAt(0)));
+    return strBuilder.toString();
+  }
+
   public boolean isMetaRegion(byte[] regionName) {
     HRegion region;
     try {
@@ -133,7 +145,7 @@ class QosFunction implements Function<Pa
   public Integer apply(Pair<RequestHeader, Message> headerAndParam) {
     RequestHeader header = headerAndParam.getFirst();
     String methodName = header.getMethodName();
-    Integer priorityByAnnotation = annotatedQos.get(header.getMethodName());
+    Integer priorityByAnnotation = annotatedQos.get(methodName);
     if (priorityByAnnotation != null) {
       return priorityByAnnotation;
     }

Added: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java?rev=1519018&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java (added)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java Fri Aug 30 16:29:23 2013
@@ -0,0 +1,55 @@
+package org.apache.hadoop.hbase.regionserver;
+/**
+* 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.
+*/
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.google.protobuf.Message;
+
+/**
+ * Basic test that qos function is sort of working; i.e. a change in method naming style
+ * over in pb doesn't break it.
+ */
+@Category(SmallTests.class)
+public class TestQosFunction {
+  @Test
+  public void testPriority() {
+    HRegionServer hrs = Mockito.mock(HRegionServer.class);
+    QosFunction qosFunction = new QosFunction(hrs);
+
+    // Set method name in pb style with the method name capitalized.
+    checkMethod("ReplicateWALEntry", HConstants.REPLICATION_QOS, qosFunction);
+    // Set method name in pb style with the method name capitalized.
+    checkMethod("OpenRegion", HConstants.HIGH_QOS, qosFunction);
+  }
+
+  private void checkMethod(final String methodName, final int expected, final QosFunction qosf) {
+    RequestHeader.Builder builder = RequestHeader.newBuilder();
+    builder.setMethodName(methodName);
+    Pair<RequestHeader, Message> headerAndParam =
+      new Pair<RequestHeader, Message>(builder.build(), null);
+    assertEquals(methodName, expected, qosf.apply(headerAndParam).intValue());
+  }
+}
\ No newline at end of file