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
21 import java.util.Comparator;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.Abortable;
27 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
32
33
34
35
36
37 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
38 @InterfaceStability.Evolving
39 public class SimpleRpcScheduler extends RpcScheduler {
40 public static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
41
42 public static final String CALL_QUEUE_READ_SHARE_CONF_KEY =
43 "hbase.ipc.server.callqueue.read.ratio";
44 public static final String CALL_QUEUE_SCAN_SHARE_CONF_KEY =
45 "hbase.ipc.server.callqueue.scan.ratio";
46 public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY =
47 "hbase.ipc.server.callqueue.handler.factor";
48
49
50
51
52
53
54 public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
55 public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
56 public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
57 public static final String CALL_QUEUE_TYPE_CONF_DEFAULT = CALL_QUEUE_TYPE_FIFO_CONF_VALUE;
58
59
60 public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY
61 = "hbase.ipc.server.queue.max.call.delay";
62
63
64
65
66
67
68
69
70 private static class CallPriorityComparator implements Comparator<CallRunner> {
71 private final static int DEFAULT_MAX_CALL_DELAY = 5000;
72
73 private final PriorityFunction priority;
74 private final int maxDelay;
75
76 public CallPriorityComparator(final Configuration conf, final PriorityFunction priority) {
77 this.priority = priority;
78 this.maxDelay = conf.getInt(QUEUE_MAX_CALL_DELAY_CONF_KEY, DEFAULT_MAX_CALL_DELAY);
79 }
80
81 @Override
82 public int compare(CallRunner a, CallRunner b) {
83 RpcServer.Call callA = a.getCall();
84 RpcServer.Call callB = b.getCall();
85 long deadlineA = priority.getDeadline(callA.getHeader(), callA.param);
86 long deadlineB = priority.getDeadline(callB.getHeader(), callB.param);
87 deadlineA = callA.timestamp + Math.min(deadlineA, maxDelay);
88 deadlineB = callB.timestamp + Math.min(deadlineB, maxDelay);
89 return (int)(deadlineA - deadlineB);
90 }
91 }
92
93 private int port;
94 private final PriorityFunction priority;
95 private final RpcExecutor callExecutor;
96 private final RpcExecutor priorityExecutor;
97 private final RpcExecutor replicationExecutor;
98
99
100 private final int highPriorityLevel;
101
102 private Abortable abortable = null;
103
104
105
106
107
108
109
110
111
112 public SimpleRpcScheduler(
113 Configuration conf,
114 int handlerCount,
115 int priorityHandlerCount,
116 int replicationHandlerCount,
117 PriorityFunction priority,
118 Abortable server,
119 int highPriorityLevel) {
120 int maxQueueLength = conf.getInt("hbase.ipc.server.max.callqueue.length",
121 handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
122 this.priority = priority;
123 this.highPriorityLevel = highPriorityLevel;
124 this.abortable = server;
125
126 String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY, CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
127 float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
128 float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
129
130 float callQueuesHandlersFactor = conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0);
131 int numCallQueues = Math.max(1, (int)Math.round(handlerCount * callQueuesHandlersFactor));
132
133 LOG.info("Using " + callQueueType + " as user call queue, count=" + numCallQueues);
134
135 if (numCallQueues > 1 && callqReadShare > 0) {
136
137 if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
138 CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
139 callExecutor = new RWQueueRpcExecutor("RW.deadline.Q", handlerCount, numCallQueues,
140 callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
141 BoundedPriorityBlockingQueue.class, callPriority);
142 } else {
143 callExecutor = new RWQueueRpcExecutor("RW.fifo.Q", handlerCount, numCallQueues,
144 callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
145 }
146 } else {
147
148 if (callQueueType.equals(CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE)) {
149 CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
150 callExecutor = new BalancedQueueRpcExecutor("B.deadline.Q", handlerCount, numCallQueues,
151 conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
152 } else {
153 callExecutor = new BalancedQueueRpcExecutor("B.fifo.Q", handlerCount,
154 numCallQueues, maxQueueLength, conf, abortable);
155 }
156 }
157
158
159 this.priorityExecutor = priorityHandlerCount > 0 ?
160 new BalancedQueueRpcExecutor("B.priority.fifo.Q", priorityHandlerCount, 2, maxQueueLength) : null;
161
162 this.replicationExecutor =
163 replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("B.replication.fifo.Q",
164 replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
165 }
166
167 public SimpleRpcScheduler(
168 Configuration conf,
169 int handlerCount,
170 int priorityHandlerCount,
171 int replicationHandlerCount,
172 PriorityFunction priority,
173 int highPriorityLevel) {
174 this(conf, handlerCount, priorityHandlerCount, replicationHandlerCount, priority,
175 null, highPriorityLevel);
176 }
177
178 @Override
179 public void init(Context context) {
180 this.port = context.getListenerAddress().getPort();
181 }
182
183 @Override
184 public void start() {
185 callExecutor.start(port);
186 if (priorityExecutor != null) priorityExecutor.start(port);
187 if (replicationExecutor != null) replicationExecutor.start(port);
188 }
189
190 @Override
191 public void stop() {
192 callExecutor.stop();
193 if (priorityExecutor != null) priorityExecutor.stop();
194 if (replicationExecutor != null) replicationExecutor.stop();
195 }
196
197 @Override
198 public void dispatch(CallRunner callTask) throws InterruptedException {
199 RpcServer.Call call = callTask.getCall();
200 int level = priority.getPriority(call.getHeader(), call.param);
201 if (priorityExecutor != null && level > highPriorityLevel) {
202 priorityExecutor.dispatch(callTask);
203 } else if (replicationExecutor != null && level == HConstants.REPLICATION_QOS) {
204 replicationExecutor.dispatch(callTask);
205 } else {
206 callExecutor.dispatch(callTask);
207 }
208 }
209
210 @Override
211 public int getGeneralQueueLength() {
212 return callExecutor.getQueueLength();
213 }
214
215 @Override
216 public int getPriorityQueueLength() {
217 return priorityExecutor == null ? 0 : priorityExecutor.getQueueLength();
218 }
219
220 @Override
221 public int getReplicationQueueLength() {
222 return replicationExecutor == null ? 0 : replicationExecutor.getQueueLength();
223 }
224
225 @Override
226 public int getActiveRpcHandlerCount() {
227 return callExecutor.getActiveHandlerCount() +
228 (priorityExecutor == null ? 0 : priorityExecutor.getActiveHandlerCount()) +
229 (replicationExecutor == null ? 0 : replicationExecutor.getActiveHandlerCount());
230 }
231 }
232