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  package org.apache.hadoop.hbase.ipc;
19  
20  import java.util.List;
21  import java.util.concurrent.BlockingQueue;
22  import java.util.concurrent.LinkedBlockingQueue;
23  import java.util.concurrent.atomic.AtomicInteger;
24  
25  import org.apache.hadoop.classification.InterfaceAudience;
26  import org.apache.hadoop.classification.InterfaceStability;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.hbase.HConstants;
29  
30  import com.google.common.base.Strings;
31  import com.google.common.collect.Lists;
32  
33  /**
34   * A scheduler that maintains isolated handler pools for general, high-priority and replication
35   * requests.
36   */
37  @InterfaceAudience.Private
38  @InterfaceStability.Evolving
39  public class SimpleRpcScheduler implements RpcScheduler {
40  
41    private int port;
42    private final int handlerCount;
43    private final int priorityHandlerCount;
44    private final int replicationHandlerCount;
45    private final PriorityFunction priority;
46    final BlockingQueue<CallRunner> callQueue;
47    final BlockingQueue<CallRunner> priorityCallQueue;
48    final BlockingQueue<CallRunner> replicationQueue;
49    private volatile boolean running = false;
50    private final List<Thread> handlers = Lists.newArrayList();
51    private AtomicInteger activeHandlerCount = new AtomicInteger(0);
52    /** What level a high priority call is at. */
53    private final int highPriorityLevel;
54  
55    /**
56     * @param conf
57     * @param handlerCount the number of handler threads that will be used to process calls
58     * @param priorityHandlerCount How many threads for priority handling.
59     * @param replicationHandlerCount How many threads for replication handling.
60     * @param highPriorityLevel
61     * @param priority Function to extract request priority.
62     */
63    public SimpleRpcScheduler(
64        Configuration conf,
65        int handlerCount,
66        int priorityHandlerCount,
67        int replicationHandlerCount,
68        PriorityFunction priority,
69        int highPriorityLevel) {
70      int maxQueueLength = conf.getInt("ipc.server.max.callqueue.length",
71          handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
72      this.handlerCount = handlerCount;
73      this.priorityHandlerCount = priorityHandlerCount;
74      this.replicationHandlerCount = replicationHandlerCount;
75      this.priority = priority;
76      this.highPriorityLevel = highPriorityLevel;
77      this.callQueue = new LinkedBlockingQueue<CallRunner>(maxQueueLength);
78      this.priorityCallQueue = priorityHandlerCount > 0
79          ? new LinkedBlockingQueue<CallRunner>(maxQueueLength)
80          : null;
81      this.replicationQueue = replicationHandlerCount > 0
82          ? new LinkedBlockingQueue<CallRunner>(maxQueueLength)
83          : null;
84    }
85  
86    @Override
87    public void init(Context context) {
88      this.port = context.getListenerAddress().getPort();
89    }
90  
91    @Override
92    public void start() {
93      running = true;
94      startHandlers(handlerCount, callQueue, null);
95      if (priorityCallQueue != null) {
96        startHandlers(priorityHandlerCount, priorityCallQueue, "Priority.");
97      }
98      if (replicationQueue != null) {
99        startHandlers(replicationHandlerCount, replicationQueue, "Replication.");
100     }
101   }
102 
103   private void startHandlers(
104       int handlerCount,
105       final BlockingQueue<CallRunner> callQueue,
106       String threadNamePrefix) {
107     for (int i = 0; i < handlerCount; i++) {
108       Thread t = new Thread(new Runnable() {
109         @Override
110         public void run() {
111           consumerLoop(callQueue);
112         }
113       });
114       t.setDaemon(true);
115       t.setName(Strings.nullToEmpty(threadNamePrefix) + "RpcServer.handler=" + i + ",port=" + port);
116       t.start();
117       handlers.add(t);
118     }
119   }
120 
121   @Override
122   public void stop() {
123     running = false;
124     for (Thread handler : handlers) {
125       handler.interrupt();
126     }
127   }
128 
129   @Override
130   public void dispatch(CallRunner callTask) throws InterruptedException {
131     RpcServer.Call call = callTask.getCall();
132     int level = priority.getPriority(call.header, call.param);
133     if (priorityCallQueue != null && level > highPriorityLevel) {
134       priorityCallQueue.put(callTask);
135     } else if (replicationQueue != null && level == HConstants.REPLICATION_QOS) {
136       replicationQueue.put(callTask);
137     } else {
138       callQueue.put(callTask); // queue the call; maybe blocked here
139     }
140   }
141 
142   @Override
143   public int getGeneralQueueLength() {
144     return callQueue.size();
145   }
146 
147   @Override
148   public int getPriorityQueueLength() {
149     return priorityCallQueue == null ? 0 : priorityCallQueue.size();
150   }
151 
152   @Override
153   public int getReplicationQueueLength() {
154     return replicationQueue == null ? 0 : replicationQueue.size();
155   }
156 
157   @Override
158   public int getActiveRpcHandlerCount() {
159     return activeHandlerCount.get();
160   }
161 
162   private void consumerLoop(BlockingQueue<CallRunner> myQueue) {
163     while (running) {
164       try {
165         CallRunner task = myQueue.take();
166         try {
167           activeHandlerCount.incrementAndGet();
168           task.run();
169         } finally {
170           activeHandlerCount.decrementAndGet();
171         }
172       } catch (InterruptedException e) {
173         Thread.currentThread().interrupt();
174       }
175     }
176   }
177 }
178