View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.ipc;
20  
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.concurrent.BlockingQueue;
24  import java.util.concurrent.LinkedBlockingQueue;
25  
26  import org.apache.commons.lang.ArrayUtils;
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.classification.InterfaceStability;
31  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
32  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
33  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
34  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
35  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
36  import org.apache.hadoop.hbase.util.ReflectionUtils;
37  
38  import com.google.protobuf.Message;
39  
40  /**
41   * RPC Executor that uses different queues for reads and writes.
42   * Each handler has its own queue and there is no stealing.
43   */
44  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
45  @InterfaceStability.Evolving
46  public class RWQueueRpcExecutor extends RpcExecutor {
47    private static final Log LOG = LogFactory.getLog(RWQueueRpcExecutor.class);
48  
49    private final List<BlockingQueue<CallRunner>> queues;
50    private final QueueBalancer writeBalancer;
51    private final QueueBalancer readBalancer;
52    private final int writeHandlersCount;
53    private final int readHandlersCount;
54    private final int numWriteQueues;
55    private final int numReadQueues;
56  
57    public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
58        final float readShare, final int maxQueueLength) {
59      this(name, handlerCount, numQueues, readShare, maxQueueLength,
60        LinkedBlockingQueue.class);
61    }
62  
63    public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
64        final float readShare, final int maxQueueLength,
65        final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
66      this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
67        calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare),
68        LinkedBlockingQueue.class, new Object[] {maxQueueLength},
69        readQueueClass, ArrayUtils.addAll(new Object[] {maxQueueLength}, readQueueInitArgs));
70    }
71  
72    public RWQueueRpcExecutor(final String name, final int writeHandlers, final int readHandlers,
73        final int numWriteQueues, final int numReadQueues,
74        final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
75        final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
76      super(name, Math.max(writeHandlers + readHandlers, numWriteQueues + numReadQueues));
77  
78      this.writeHandlersCount = Math.max(writeHandlers, numWriteQueues);
79      this.readHandlersCount = Math.max(readHandlers, numReadQueues);
80      this.numWriteQueues = numWriteQueues;
81      this.numReadQueues = numReadQueues;
82      this.writeBalancer = getBalancer(numWriteQueues);
83      this.readBalancer = getBalancer(numReadQueues);
84  
85      queues = new ArrayList<BlockingQueue<CallRunner>>(writeHandlersCount + readHandlersCount);
86      LOG.debug(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount +
87                " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount);
88  
89      for (int i = 0; i < numWriteQueues; ++i) {
90        queues.add((BlockingQueue<CallRunner>)
91          ReflectionUtils.newInstance(writeQueueClass, writeQueueInitArgs));
92      }
93  
94      for (int i = 0; i < numReadQueues; ++i) {
95        queues.add((BlockingQueue<CallRunner>)
96          ReflectionUtils.newInstance(readQueueClass, readQueueInitArgs));
97      }
98    }
99  
100   @Override
101   protected void startHandlers(final int port) {
102     startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port);
103     startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port);
104   }
105 
106   @Override
107   public void dispatch(final CallRunner callTask) throws InterruptedException {
108     RpcServer.Call call = callTask.getCall();
109     int queueIndex;
110     if (isWriteRequest(call.getHeader(), call.param)) {
111       queueIndex = writeBalancer.getNextQueue();
112     } else {
113       queueIndex = numWriteQueues + readBalancer.getNextQueue();
114     }
115     queues.get(queueIndex).put(callTask);
116   }
117 
118   private boolean isWriteRequest(final RequestHeader header, final Message param) {
119     // TODO: Is there a better way to do this?
120     String methodName = header.getMethodName();
121     if (methodName.equalsIgnoreCase("multi") && param instanceof MultiRequest) {
122       MultiRequest multi = (MultiRequest)param;
123       for (RegionAction regionAction : multi.getRegionActionList()) {
124         for (Action action: regionAction.getActionList()) {
125           if (action.hasMutation()) {
126             return true;
127           }
128         }
129       }
130     }
131     return false;
132   }
133 
134   @Override
135   public int getQueueLength() {
136     int length = 0;
137     for (final BlockingQueue<CallRunner> queue: queues) {
138       length += queue.size();
139     }
140     return length;
141   }
142 
143   @Override
144   protected List<BlockingQueue<CallRunner>> getQueues() {
145     return queues;
146   }
147 
148   /*
149    * Calculate the number of writers based on the "total count" and the read share.
150    * You'll get at least one writer.
151    */
152   private static int calcNumWriters(final int count, final float readShare) {
153     return Math.max(1, count - Math.max(1, (int)Math.round(count * readShare)));
154   }
155 
156   /*
157    * Calculate the number of readers based on the "total count" and the read share.
158    * You'll get at least one reader.
159    */
160   private static int calcNumReaders(final int count, final float readShare) {
161     return count - calcNumWriters(count, readShare);
162   }
163 }