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.List;
22  import java.util.concurrent.BlockingQueue;
23  import java.util.concurrent.LinkedBlockingQueue;
24  import java.util.concurrent.atomic.AtomicInteger;
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.conf.Configuration;
30  import org.apache.hadoop.hbase.Abortable;
31  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.classification.InterfaceStability;
34  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
35  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
36  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
37  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
38  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
39  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
40  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
41  import org.apache.hadoop.hbase.util.ReflectionUtils;
42  
43  import com.google.protobuf.Message;
44  
45  /**
46   * RPC Executor that uses different queues for reads and writes.
47   * With the options to use different queues/executors for gets and scans.
48   * Each handler has its own queue and there is no stealing.
49   */
50  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
51  @InterfaceStability.Evolving
52  public class RWQueueRpcExecutor extends RpcExecutor {
53    private static final Log LOG = LogFactory.getLog(RWQueueRpcExecutor.class);
54  
55    public static final String CALL_QUEUE_READ_SHARE_CONF_KEY =
56        "hbase.ipc.server.callqueue.read.ratio";
57    public static final String CALL_QUEUE_SCAN_SHARE_CONF_KEY =
58        "hbase.ipc.server.callqueue.scan.ratio";
59  
60    private final QueueBalancer writeBalancer;
61    private final QueueBalancer readBalancer;
62    private final QueueBalancer scanBalancer;
63    private final int writeHandlersCount;
64    private final int readHandlersCount;
65    private final int scanHandlersCount;
66    private final int numWriteQueues;
67    private final int numReadQueues;
68    private final int numScanQueues;
69  
70    private final AtomicInteger activeWriteHandlerCount = new AtomicInteger(0);
71    private final AtomicInteger activeReadHandlerCount = new AtomicInteger(0);
72    private final AtomicInteger activeScanHandlerCount = new AtomicInteger(0);
73  
74    public RWQueueRpcExecutor(final String name, final int handlerCount, final int maxQueueLength,
75        final PriorityFunction priority, final Configuration conf, final Abortable abortable) {
76      super(name, handlerCount, maxQueueLength, priority, conf, abortable);
77  
78      float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
79      float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
80  
81      numWriteQueues = calcNumWriters(this.numCallQueues, callqReadShare);
82      writeHandlersCount = Math.max(numWriteQueues, calcNumWriters(handlerCount, callqReadShare));
83  
84      int readQueues = calcNumReaders(this.numCallQueues, callqReadShare);
85      int readHandlers = Math.max(readQueues, calcNumReaders(handlerCount, callqReadShare));
86  
87      int scanQueues = Math.max(0, (int)Math.floor(readQueues * callqScanShare));
88      int scanHandlers = Math.max(0, (int)Math.floor(readHandlers * callqScanShare));
89  
90      if ((readQueues - scanQueues) > 0) {
91        readQueues -= scanQueues;
92        readHandlers -= scanHandlers;
93      } else {
94        scanQueues = 0;
95        scanHandlers = 0;
96      }
97  
98      numReadQueues = readQueues;
99      readHandlersCount = readHandlers;
100     numScanQueues = scanQueues;
101     scanHandlersCount = scanHandlers;
102 
103     this.writeBalancer = getBalancer(numWriteQueues);
104     this.readBalancer = getBalancer(numReadQueues);
105     this.scanBalancer = numScanQueues > 0 ? getBalancer(numScanQueues) : null;
106 
107     initializeQueues(numWriteQueues);
108     initializeQueues(numReadQueues);
109     initializeQueues(numScanQueues);
110 
111     LOG.info(getName() + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount
112       + " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount + " scanQueues="
113       + numScanQueues + " scanHandlers=" + scanHandlersCount);
114   }
115 
116   @Deprecated
117   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
118       final float readShare, final int maxQueueLength,
119       final Configuration conf, final Abortable abortable) {
120     this(name, handlerCount, numQueues, readShare, maxQueueLength, 0,
121       conf, abortable, LinkedBlockingQueue.class);
122   }
123 
124   @Deprecated
125   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
126       final float readShare, final float scanShare, final int maxQueueLength) {
127     this(name, handlerCount, numQueues, readShare, scanShare, maxQueueLength, null, null);
128   }
129 
130   @Deprecated
131   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
132       final float readShare, final float scanShare, final int maxQueueLength,
133       final Configuration conf, final Abortable abortable) {
134     this(name, handlerCount, numQueues, readShare, scanShare, maxQueueLength,
135       conf, abortable, LinkedBlockingQueue.class);
136   }
137 
138   @Deprecated
139   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
140       final float readShare, final int maxQueueLength,
141       final Configuration conf, final Abortable abortable,
142       final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
143     this(name, handlerCount, numQueues, readShare, 0, maxQueueLength, conf, abortable,
144       readQueueClass, readQueueInitArgs);
145   }
146 
147   @Deprecated
148   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
149       final float readShare, final float scanShare, final int maxQueueLength,
150       final Configuration conf, final Abortable abortable,
151       final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
152     this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
153       calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare), scanShare,
154       LinkedBlockingQueue.class, new Object[] {maxQueueLength},
155       readQueueClass, ArrayUtils.addAll(new Object[] {maxQueueLength}, readQueueInitArgs));
156   }
157 
158   @Deprecated
159   public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
160       final float readShare, final float scanShare,
161       final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
162       final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
163     this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
164       calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare), scanShare,
165       writeQueueClass, writeQueueInitArgs,
166       readQueueClass, readQueueInitArgs);
167   }
168 
169   @Deprecated
170   public RWQueueRpcExecutor(final String name, final int writeHandlers, final int readHandlers,
171       final int numWriteQueues, final int numReadQueues,
172       final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
173       final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
174     this(name, writeHandlers, readHandlers, numWriteQueues, numReadQueues, 0,
175       writeQueueClass, writeQueueInitArgs, readQueueClass, readQueueInitArgs);
176   }
177 
178   @Deprecated
179   public RWQueueRpcExecutor(final String name, int writeHandlers, int readHandlers,
180       int numWriteQueues, int numReadQueues, float scanShare,
181       final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
182       final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
183     super(name, Math.max(writeHandlers, numWriteQueues) + Math.max(readHandlers, numReadQueues),
184         numWriteQueues + numReadQueues);
185 
186     this.writeHandlersCount = Math.max(writeHandlers, numWriteQueues);
187     this.numWriteQueues = numWriteQueues;
188 
189     int numScanQueues = Math.max(0, (int) Math.floor(numReadQueues * scanShare));
190     int scanHandlers = Math.max(0, (int) Math.floor(readHandlers * scanShare));
191     if ((numReadQueues - numScanQueues) > 0) {
192       numReadQueues -= numScanQueues;
193       readHandlers -= scanHandlers;
194     } else {
195       numScanQueues = 0;
196       scanHandlers = 0;
197     }
198 
199     this.readHandlersCount = Math.max(readHandlers, numReadQueues);
200     this.scanHandlersCount = Math.max(scanHandlers, numScanQueues);
201     this.numReadQueues = numReadQueues;
202     this.numScanQueues = numScanQueues;
203 
204     this.writeBalancer = getBalancer(numWriteQueues);
205     this.readBalancer = getBalancer(numReadQueues);
206     this.scanBalancer = numScanQueues > 0 ? getBalancer(numScanQueues) : null;
207 
208     LOG.info(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount
209         + " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount + " scanQueues="
210         + numScanQueues + " scanHandlers=" + scanHandlersCount);
211 
212     if (writeQueueInitArgs.length > 0) {
213       currentQueueLimit = (int) writeQueueInitArgs[0];
214       writeQueueInitArgs[0] = Math.max((int) writeQueueInitArgs[0],
215         DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
216     }
217     for (int i = 0; i < numWriteQueues; ++i) {
218       queues.add((BlockingQueue<CallRunner>) ReflectionUtils.newInstance(writeQueueClass,
219         writeQueueInitArgs));
220     }
221 
222     if (readQueueInitArgs.length > 0) {
223       currentQueueLimit = (int) readQueueInitArgs[0];
224       readQueueInitArgs[0] = Math.max((int) readQueueInitArgs[0],
225         DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT);
226     }
227     for (int i = 0; i < (numReadQueues + numScanQueues); ++i) {
228       queues.add((BlockingQueue<CallRunner>) ReflectionUtils.newInstance(readQueueClass,
229         readQueueInitArgs));
230     }
231   }
232 
233   @Override
234   protected int computeNumCallQueues(final int handlerCount, final float callQueuesHandlersFactor) {
235     // at least 1 read queue and 1 write queue
236     return Math.max(2, (int) Math.round(handlerCount * callQueuesHandlersFactor));
237   }
238 
239   @Override
240   protected void startHandlers(final int port) {
241     startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port,
242       activeWriteHandlerCount);
243     startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port,
244       activeReadHandlerCount);
245     if (numScanQueues > 0) {
246       startHandlers(".scan", scanHandlersCount, queues, numWriteQueues + numReadQueues,
247         numScanQueues, port, activeScanHandlerCount);
248     }
249   }
250 
251   @Override
252   public boolean dispatch(final CallRunner callTask) throws InterruptedException {
253     RpcServer.Call call = callTask.getCall();
254     int queueIndex;
255     if (isWriteRequest(call.getHeader(), call.param)) {
256       queueIndex = writeBalancer.getNextQueue();
257     } else if (numScanQueues > 0 && isScanRequest(call.getHeader(), call.param)) {
258       queueIndex = numWriteQueues + numReadQueues + scanBalancer.getNextQueue();
259     } else {
260       queueIndex = numWriteQueues + readBalancer.getNextQueue();
261     }
262 
263     BlockingQueue<CallRunner> queue = queues.get(queueIndex);
264     if (queue.size() >= currentQueueLimit) {
265       return false;
266     }
267     return queue.offer(callTask);
268   }
269 
270   @Override
271   public int getWriteQueueLength() {
272     int length = 0;
273     for (int i = 0; i < numWriteQueues; i++) {
274       length += queues.get(i).size();
275     }
276     return length;
277   }
278 
279   @Override
280   public int getReadQueueLength() {
281     int length = 0;
282     for (int i = numWriteQueues; i < (numWriteQueues + numReadQueues); i++) {
283       length += queues.get(i).size();
284     }
285     return length;
286   }
287 
288   @Override
289   public int getScanQueueLength() {
290     int length = 0;
291     for (int i = numWriteQueues + numReadQueues;
292         i < (numWriteQueues + numReadQueues + numScanQueues); i++) {
293       length += queues.get(i).size();
294     }
295     return length;
296   }
297 
298   @Override
299   public int getActiveHandlerCount() {
300     return activeWriteHandlerCount.get() + activeReadHandlerCount.get()
301         + activeScanHandlerCount.get();
302   }
303 
304   @Override
305   public int getActiveWriteHandlerCount() {
306     return activeWriteHandlerCount.get();
307   }
308 
309   @Override
310   public int getActiveReadHandlerCount() {
311     return activeReadHandlerCount.get();
312   }
313 
314   @Override
315   public int getActiveScanHandlerCount() {
316     return activeScanHandlerCount.get();
317   }
318 
319   private boolean isWriteRequest(final RequestHeader header, final Message param) {
320     // TODO: Is there a better way to do this?
321     if (param instanceof MultiRequest) {
322       MultiRequest multi = (MultiRequest)param;
323       for (RegionAction regionAction : multi.getRegionActionList()) {
324         for (Action action: regionAction.getActionList()) {
325           if (action.hasMutation()) {
326             return true;
327           }
328         }
329       }
330     }
331     if (param instanceof MutateRequest) {
332       return true;
333     }
334     // Below here are methods for master. It's a pretty brittle version of this.
335     // Not sure that master actually needs a read/write queue since 90% of requests to
336     // master are writing to status or changing the meta table.
337     // All other read requests are admin generated and can be processed whenever.
338     // However changing that would require a pretty drastic change and should be done for
339     // the next major release and not as a fix for HBASE-14239
340     if (param instanceof RegionServerStatusProtos.ReportRegionStateTransitionRequest) {
341       return true;
342     }
343     if (param instanceof RegionServerStatusProtos.RegionServerStartupRequest) {
344       return true;
345     }
346     if (param instanceof RegionServerStatusProtos.RegionServerReportRequest) {
347       return true;
348     }
349     return false;
350   }
351 
352   private boolean isScanRequest(final RequestHeader header, final Message param) {
353     return param instanceof ScanRequest;
354   }
355 
356   /*
357    * Calculate the number of writers based on the "total count" and the read share.
358    * You'll get at least one writer.
359    */
360   private static int calcNumWriters(final int count, final float readShare) {
361     return Math.max(1, count - Math.max(1, (int)Math.round(count * readShare)));
362   }
363 
364   /*
365    * Calculate the number of readers based on the "total count" and the read share.
366    * You'll get at least one reader.
367    */
368   private static int calcNumReaders(final int count, final float readShare) {
369     return count - calcNumWriters(count, readShare);
370   }
371 }