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