1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.classification.InterfaceStability;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.HConstants;
28
29 import com.google.common.base.Strings;
30 import com.google.common.collect.Lists;
31
32
33
34
35
36 @InterfaceAudience.Private
37 @InterfaceStability.Evolving
38 public class SimpleRpcScheduler implements RpcScheduler {
39
40 private int port;
41 private final int handlerCount;
42 private final int priorityHandlerCount;
43 private final int replicationHandlerCount;
44 private final PriorityFunction priority;
45 final BlockingQueue<CallRunner> callQueue;
46 final BlockingQueue<CallRunner> priorityCallQueue;
47 final BlockingQueue<CallRunner> replicationQueue;
48 private volatile boolean running = false;
49 private final List<Thread> handlers = Lists.newArrayList();
50
51
52 private final int highPriorityLevel;
53
54
55
56
57
58
59
60
61
62 public SimpleRpcScheduler(
63 Configuration conf,
64 int handlerCount,
65 int priorityHandlerCount,
66 int replicationHandlerCount,
67 PriorityFunction priority,
68 int highPriorityLevel) {
69 int maxQueueLength = conf.getInt("ipc.server.max.callqueue.length",
70 handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
71 this.handlerCount = handlerCount;
72 this.priorityHandlerCount = priorityHandlerCount;
73 this.replicationHandlerCount = replicationHandlerCount;
74 this.priority = priority;
75 this.highPriorityLevel = highPriorityLevel;
76 this.callQueue = new LinkedBlockingQueue<CallRunner>(maxQueueLength);
77 this.priorityCallQueue = priorityHandlerCount > 0
78 ? new LinkedBlockingQueue<CallRunner>(maxQueueLength)
79 : null;
80 this.replicationQueue = replicationHandlerCount > 0
81 ? new LinkedBlockingQueue<CallRunner>(maxQueueLength)
82 : null;
83 }
84
85 @Override
86 public void init(Context context) {
87 this.port = context.getListenerAddress().getPort();
88 }
89
90 @Override
91 public void start() {
92 running = true;
93 startHandlers(handlerCount, callQueue, null);
94 if (priorityCallQueue != null) {
95 startHandlers(priorityHandlerCount, priorityCallQueue, "Priority.");
96 }
97 if (replicationQueue != null) {
98 startHandlers(replicationHandlerCount, replicationQueue, "Replication.");
99 }
100 }
101
102 private void startHandlers(
103 int handlerCount,
104 final BlockingQueue<CallRunner> callQueue,
105 String threadNamePrefix) {
106 for (int i = 0; i < handlerCount; i++) {
107 Thread t = new Thread(new Runnable() {
108 @Override
109 public void run() {
110 consumerLoop(callQueue);
111 }
112 });
113 t.setDaemon(true);
114 t.setName(Strings.nullToEmpty(threadNamePrefix) + "RpcServer.handler=" + i + ",port=" + port);
115 t.start();
116 handlers.add(t);
117 }
118 }
119
120 @Override
121 public void stop() {
122 running = false;
123 for (Thread handler : handlers) {
124 handler.interrupt();
125 }
126 }
127
128 @Override
129 public void dispatch(CallRunner callTask) throws InterruptedException {
130 RpcServer.Call call = callTask.getCall();
131 int level = priority.getPriority(call.header, call.param);
132 if (priorityCallQueue != null && level > highPriorityLevel) {
133 priorityCallQueue.put(callTask);
134 } else if (replicationQueue != null && level == HConstants.REPLICATION_QOS) {
135 replicationQueue.put(callTask);
136 } else {
137 callQueue.put(callTask);
138 }
139 }
140
141 @Override
142 public int getGeneralQueueLength() {
143 return callQueue.size();
144 }
145
146 @Override
147 public int getPriorityQueueLength() {
148 return priorityCallQueue == null ? 0 : priorityCallQueue.size();
149 }
150
151 @Override
152 public int getReplicationQueueLength() {
153 return replicationQueue == null ? 0 : replicationQueue.size();
154 }
155
156 private void consumerLoop(BlockingQueue<CallRunner> myQueue) {
157 while (running) {
158 try {
159 CallRunner task = myQueue.take();
160 task.run();
161 } catch (InterruptedException e) {
162 Thread.currentThread().interrupt();
163 }
164 }
165 }
166 }
167