1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.ipc;
20
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.concurrent.BlockingQueue;
24 import java.util.concurrent.LinkedBlockingQueue;
25
26 import org.apache.commons.lang.ArrayUtils;
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.Abortable;
31 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.classification.InterfaceStability;
34 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
35 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
36 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
37 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
38 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
39 import org.apache.hadoop.hbase.util.ReflectionUtils;
40
41 import com.google.protobuf.Message;
42
43
44
45
46
47
48 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
49 @InterfaceStability.Evolving
50 public class RWQueueRpcExecutor extends RpcExecutor {
51 private static final Log LOG = LogFactory.getLog(RWQueueRpcExecutor.class);
52
53 private final List<BlockingQueue<CallRunner>> queues;
54 private final QueueBalancer writeBalancer;
55 private final QueueBalancer readBalancer;
56 private final QueueBalancer scanBalancer;
57 private final int writeHandlersCount;
58 private final int readHandlersCount;
59 private final int scanHandlersCount;
60 private final int numWriteQueues;
61 private final int numReadQueues;
62 private final int numScanQueues;
63
64 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
65 final float readShare, final int maxQueueLength,
66 final Configuration conf, final Abortable abortable) {
67 this(name, handlerCount, numQueues, readShare, maxQueueLength, 0,
68 conf, abortable, LinkedBlockingQueue.class);
69 }
70
71 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
72 final float readShare, final float scanShare, final int maxQueueLength) {
73 this(name, handlerCount, numQueues, readShare, scanShare, maxQueueLength, null, null);
74 }
75
76 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
77 final float readShare, final float scanShare, final int maxQueueLength,
78 final Configuration conf, final Abortable abortable) {
79 this(name, handlerCount, numQueues, readShare, scanShare, maxQueueLength,
80 conf, abortable, LinkedBlockingQueue.class);
81 }
82
83 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
84 final float readShare, final int maxQueueLength,
85 final Configuration conf, final Abortable abortable,
86 final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
87 this(name, handlerCount, numQueues, readShare, 0, maxQueueLength, conf, abortable,
88 readQueueClass, readQueueInitArgs);
89 }
90
91 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
92 final float readShare, final float scanShare, final int maxQueueLength,
93 final Configuration conf, final Abortable abortable,
94 final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
95 this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
96 calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare), scanShare,
97 LinkedBlockingQueue.class, new Object[] {maxQueueLength},
98 readQueueClass, ArrayUtils.addAll(new Object[] {maxQueueLength}, readQueueInitArgs));
99 }
100
101 public RWQueueRpcExecutor(final String name, final int writeHandlers, final int readHandlers,
102 final int numWriteQueues, final int numReadQueues,
103 final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
104 final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
105 this(name, writeHandlers, readHandlers, numWriteQueues, numReadQueues, 0,
106 writeQueueClass, writeQueueInitArgs, readQueueClass, readQueueInitArgs);
107 }
108
109 public RWQueueRpcExecutor(final String name, int writeHandlers, int readHandlers,
110 int numWriteQueues, int numReadQueues, float scanShare,
111 final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
112 final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
113 super(name, Math.max(writeHandlers, numWriteQueues) + Math.max(readHandlers, numReadQueues));
114
115 int numScanQueues = Math.max(0, (int)Math.floor(numReadQueues * scanShare));
116 int scanHandlers = Math.max(0, (int)Math.floor(readHandlers * scanShare));
117 if ((numReadQueues - numScanQueues) > 0) {
118 numReadQueues -= numScanQueues;
119 readHandlers -= scanHandlers;
120 } else {
121 numScanQueues = 0;
122 scanHandlers = 0;
123 }
124
125 this.writeHandlersCount = Math.max(writeHandlers, numWriteQueues);
126 this.readHandlersCount = Math.max(readHandlers, numReadQueues);
127 this.scanHandlersCount = Math.max(scanHandlers, numScanQueues);
128 this.numWriteQueues = numWriteQueues;
129 this.numReadQueues = numReadQueues;
130 this.numScanQueues = numScanQueues;
131 this.writeBalancer = getBalancer(numWriteQueues);
132 this.readBalancer = getBalancer(numReadQueues);
133 this.scanBalancer = getBalancer(numScanQueues);
134
135 queues = new ArrayList<BlockingQueue<CallRunner>>(writeHandlersCount + readHandlersCount);
136 LOG.debug(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount +
137 " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount +
138 ((numScanQueues == 0) ? "" : " scanQueues=" + numScanQueues +
139 " scanHandlers=" + scanHandlersCount));
140
141 for (int i = 0; i < numWriteQueues; ++i) {
142 queues.add((BlockingQueue<CallRunner>)
143 ReflectionUtils.newInstance(writeQueueClass, writeQueueInitArgs));
144 }
145
146 for (int i = 0; i < (numReadQueues + numScanQueues); ++i) {
147 queues.add((BlockingQueue<CallRunner>)
148 ReflectionUtils.newInstance(readQueueClass, readQueueInitArgs));
149 }
150 }
151
152 @Override
153 protected void startHandlers(final int port) {
154 startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port);
155 startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port);
156 startHandlers(".scan", scanHandlersCount, queues,
157 numWriteQueues + numReadQueues, numScanQueues, port);
158 }
159
160 @Override
161 public void dispatch(final CallRunner callTask) throws InterruptedException {
162 RpcServer.Call call = callTask.getCall();
163 int queueIndex;
164 if (isWriteRequest(call.getHeader(), call.param)) {
165 queueIndex = writeBalancer.getNextQueue();
166 } else if (numScanQueues > 0 && isScanRequest(call.getHeader(), call.param)) {
167 queueIndex = numWriteQueues + numReadQueues + scanBalancer.getNextQueue();
168 } else {
169 queueIndex = numWriteQueues + readBalancer.getNextQueue();
170 }
171 queues.get(queueIndex).put(callTask);
172 }
173
174 private boolean isWriteRequest(final RequestHeader header, final Message param) {
175
176 String methodName = header.getMethodName();
177 if (methodName.equalsIgnoreCase("multi") && param instanceof MultiRequest) {
178 MultiRequest multi = (MultiRequest)param;
179 for (RegionAction regionAction : multi.getRegionActionList()) {
180 for (Action action: regionAction.getActionList()) {
181 if (action.hasMutation()) {
182 return true;
183 }
184 }
185 }
186 }
187 if (methodName.equalsIgnoreCase("mutate")) {
188 return true;
189 }
190 return false;
191 }
192
193 private boolean isScanRequest(final RequestHeader header, final Message param) {
194 String methodName = header.getMethodName();
195 if (methodName.equalsIgnoreCase("scan")) {
196
197 ScanRequest request = (ScanRequest)param;
198 return request.hasScannerId();
199 }
200 return false;
201 }
202
203 @Override
204 public int getQueueLength() {
205 int length = 0;
206 for (final BlockingQueue<CallRunner> queue: queues) {
207 length += queue.size();
208 }
209 return length;
210 }
211
212 @Override
213 protected List<BlockingQueue<CallRunner>> getQueues() {
214 return queues;
215 }
216
217
218
219
220
221 private static int calcNumWriters(final int count, final float readShare) {
222 return Math.max(1, count - Math.max(1, (int)Math.round(count * readShare)));
223 }
224
225
226
227
228
229 private static int calcNumReaders(final int count, final float readShare) {
230 return count - calcNumWriters(count, readShare);
231 }
232 }