001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.ipc;
019
020import org.apache.hadoop.conf.Configuration;
021import org.apache.hadoop.hbase.Abortable;
022import org.apache.hadoop.hbase.HBaseInterfaceAudience;
023import org.apache.hadoop.hbase.HConstants;
024import org.apache.yetus.audience.InterfaceAudience;
025import org.apache.yetus.audience.InterfaceStability;
026import org.apache.hadoop.hbase.conf.ConfigurationObserver;
027
028/**
029 * The default scheduler. Configurable. Maintains isolated handler pools for general ('default'),
030 * high-priority ('priority'), and replication ('replication') requests. Default behavior is to
031 * balance the requests across handlers. Add configs to enable balancing by read vs writes, etc.
032 * See below article for explanation of options.
033 * @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>
034 */
035@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
036@InterfaceStability.Evolving
037public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObserver {
038  private int port;
039  private final PriorityFunction priority;
040  private final RpcExecutor callExecutor;
041  private final RpcExecutor priorityExecutor;
042  private final RpcExecutor replicationExecutor;
043
044  /** What level a high priority call is at. */
045  private final int highPriorityLevel;
046
047  private Abortable abortable = null;
048
049  /**
050   * @param conf
051   * @param handlerCount the number of handler threads that will be used to process calls
052   * @param priorityHandlerCount How many threads for priority handling.
053   * @param replicationHandlerCount How many threads for replication handling.
054   * @param highPriorityLevel
055   * @param priority Function to extract request priority.
056   */
057  public SimpleRpcScheduler(
058      Configuration conf,
059      int handlerCount,
060      int priorityHandlerCount,
061      int replicationHandlerCount,
062      PriorityFunction priority,
063      Abortable server,
064      int highPriorityLevel) {
065
066    int maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
067        handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
068    int maxPriorityQueueLength =
069        conf.getInt(RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH, maxQueueLength);
070
071    this.priority = priority;
072    this.highPriorityLevel = highPriorityLevel;
073    this.abortable = server;
074
075    String callQueueType = conf.get(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
076      RpcExecutor.CALL_QUEUE_TYPE_CONF_DEFAULT);
077    float callqReadShare = conf.getFloat(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
078
079    if (callqReadShare > 0) {
080      // at least 1 read handler and 1 write handler
081      callExecutor = new RWQueueRpcExecutor("default.RWQ", Math.max(2, handlerCount),
082        maxQueueLength, priority, conf, server);
083    } else {
084      if (RpcExecutor.isFifoQueueType(callQueueType) || RpcExecutor.isCodelQueueType(callQueueType)) {
085        callExecutor = new FastPathBalancedQueueRpcExecutor("default.FPBQ", handlerCount,
086            maxQueueLength, priority, conf, server);
087      } else {
088        callExecutor = new BalancedQueueRpcExecutor("default.BQ", handlerCount, maxQueueLength,
089            priority, conf, server);
090      }
091    }
092
093    // Create 2 queues to help priorityExecutor be more scalable.
094    this.priorityExecutor = priorityHandlerCount > 0 ? new FastPathBalancedQueueRpcExecutor(
095        "priority.FPBQ", priorityHandlerCount, RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE,
096        maxPriorityQueueLength, priority, conf, abortable) : null;
097    this.replicationExecutor = replicationHandlerCount > 0 ? new FastPathBalancedQueueRpcExecutor(
098        "replication.FPBQ", replicationHandlerCount, RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE,
099        maxQueueLength, priority, conf, abortable) : null;
100  }
101
102
103  public SimpleRpcScheduler(
104        Configuration conf,
105        int handlerCount,
106        int priorityHandlerCount,
107        int replicationHandlerCount,
108        PriorityFunction priority,
109        int highPriorityLevel) {
110    this(conf, handlerCount, priorityHandlerCount, replicationHandlerCount, priority,
111      null, highPriorityLevel);
112  }
113
114  /**
115   * Resize call queues;
116   * @param conf new configuration
117   */
118  @Override
119  public void onConfigurationChange(Configuration conf) {
120    callExecutor.resizeQueues(conf);
121    if (priorityExecutor != null) {
122      priorityExecutor.resizeQueues(conf);
123    }
124    if (replicationExecutor != null) {
125      replicationExecutor.resizeQueues(conf);
126    }
127
128    String callQueueType = conf.get(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
129      RpcExecutor.CALL_QUEUE_TYPE_CONF_DEFAULT);
130    if (RpcExecutor.isCodelQueueType(callQueueType)) {
131      callExecutor.onConfigurationChange(conf);
132    }
133  }
134
135  @Override
136  public void init(Context context) {
137    this.port = context.getListenerAddress().getPort();
138  }
139
140  @Override
141  public void start() {
142    callExecutor.start(port);
143    if (priorityExecutor != null) priorityExecutor.start(port);
144    if (replicationExecutor != null) replicationExecutor.start(port);
145  }
146
147  @Override
148  public void stop() {
149    callExecutor.stop();
150    if (priorityExecutor != null) priorityExecutor.stop();
151    if (replicationExecutor != null) replicationExecutor.stop();
152  }
153
154  @Override
155  public boolean dispatch(CallRunner callTask) throws InterruptedException {
156    RpcCall call = callTask.getRpcCall();
157    int level = priority.getPriority(call.getHeader(), call.getParam(),
158        call.getRequestUser().orElse(null));
159    if (level == HConstants.PRIORITY_UNSET) {
160      level = HConstants.NORMAL_QOS;
161    }
162    if (priorityExecutor != null && level > highPriorityLevel) {
163      return priorityExecutor.dispatch(callTask);
164    } else if (replicationExecutor != null && level == HConstants.REPLICATION_QOS) {
165      return replicationExecutor.dispatch(callTask);
166    } else {
167      return callExecutor.dispatch(callTask);
168    }
169  }
170
171  @Override
172  public int getGeneralQueueLength() {
173    return callExecutor.getQueueLength();
174  }
175
176  @Override
177  public int getPriorityQueueLength() {
178    return priorityExecutor == null ? 0 : priorityExecutor.getQueueLength();
179  }
180
181  @Override
182  public int getReplicationQueueLength() {
183    return replicationExecutor == null ? 0 : replicationExecutor.getQueueLength();
184  }
185
186  @Override
187  public int getActiveRpcHandlerCount() {
188    return callExecutor.getActiveHandlerCount() +
189           (priorityExecutor == null ? 0 : priorityExecutor.getActiveHandlerCount()) +
190           (replicationExecutor == null ? 0 : replicationExecutor.getActiveHandlerCount());
191  }
192
193  @Override
194  public long getNumGeneralCallsDropped() {
195    return callExecutor.getNumGeneralCallsDropped();
196  }
197
198  @Override
199  public long getNumLifoModeSwitches() {
200    return callExecutor.getNumLifoModeSwitches();
201  }
202
203  @Override
204  public int getWriteQueueLength() {
205    return callExecutor.getWriteQueueLength();
206  }
207
208  @Override
209  public int getReadQueueLength() {
210    return callExecutor.getReadQueueLength();
211  }
212
213  @Override
214  public int getScanQueueLength() {
215    return callExecutor.getScanQueueLength();
216  }
217
218  @Override
219  public int getActiveWriteRpcHandlerCount() {
220    return callExecutor.getActiveWriteHandlerCount();
221  }
222
223  @Override
224  public int getActiveReadRpcHandlerCount() {
225    return callExecutor.getActiveReadHandlerCount();
226  }
227
228  @Override
229  public int getActiveScanRpcHandlerCount() {
230    return callExecutor.getActiveScanHandlerCount();
231  }
232
233  @Override
234  public CallQueueInfo getCallQueueInfo() {
235    String queueName;
236
237    CallQueueInfo callQueueInfo = new CallQueueInfo();
238
239    if(null!=callExecutor) {
240      queueName = "Call Queue";
241      callQueueInfo.setCallMethodCount(queueName, callExecutor.getCallQueueCountsSummary());
242      callQueueInfo.setCallMethodSize(queueName, callExecutor.getCallQueueSizeSummary());
243    }
244
245    if(null!=priorityExecutor) {
246      queueName = "Priority Queue";
247      callQueueInfo.setCallMethodCount(queueName, priorityExecutor.getCallQueueCountsSummary());
248      callQueueInfo.setCallMethodSize(queueName, priorityExecutor.getCallQueueSizeSummary());
249    }
250
251    if(null!=replicationExecutor) {
252      queueName = "Replication Queue";
253      callQueueInfo.setCallMethodCount(queueName, replicationExecutor.getCallQueueCountsSummary());
254      callQueueInfo.setCallMethodSize(queueName, replicationExecutor.getCallQueueSizeSummary());
255    }
256
257    return callQueueInfo;
258  }
259
260}
261