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.RPCProtos.RequestHeader;
38 import org.apache.hadoop.hbase.util.ReflectionUtils;
39
40 import com.google.protobuf.Message;
41
42
43
44
45
46 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
47 @InterfaceStability.Evolving
48 public class RWQueueRpcExecutor extends RpcExecutor {
49 private static final Log LOG = LogFactory.getLog(RWQueueRpcExecutor.class);
50
51 private final List<BlockingQueue<CallRunner>> queues;
52 private final QueueBalancer writeBalancer;
53 private final QueueBalancer readBalancer;
54 private final int writeHandlersCount;
55 private final int readHandlersCount;
56 private final int numWriteQueues;
57 private final int numReadQueues;
58
59 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
60 final float readShare, final int maxQueueLength) {
61 this(name, handlerCount, numQueues, readShare, maxQueueLength, null, null);
62 }
63
64 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
65 final float readShare, final int maxQueueLength, final Configuration conf, final Abortable abortable) {
66 this(name, handlerCount, numQueues, readShare, maxQueueLength, conf, abortable,
67 LinkedBlockingQueue.class);
68 }
69
70 public RWQueueRpcExecutor(final String name, final int handlerCount, final int numQueues,
71 final float readShare, final int maxQueueLength,
72 final Configuration conf, final Abortable abortable,
73 final Class<? extends BlockingQueue> readQueueClass, Object... readQueueInitArgs) {
74 this(name, calcNumWriters(handlerCount, readShare), calcNumReaders(handlerCount, readShare),
75 calcNumWriters(numQueues, readShare), calcNumReaders(numQueues, readShare),
76 conf, abortable,
77 LinkedBlockingQueue.class, new Object[] {maxQueueLength},
78 readQueueClass, ArrayUtils.addAll(new Object[] {maxQueueLength}, readQueueInitArgs));
79 }
80
81 public RWQueueRpcExecutor(final String name, final int writeHandlers, final int readHandlers,
82 final int numWriteQueues, final int numReadQueues,
83 final Configuration conf, final Abortable abortable,
84 final Class<? extends BlockingQueue> writeQueueClass, Object[] writeQueueInitArgs,
85 final Class<? extends BlockingQueue> readQueueClass, Object[] readQueueInitArgs) {
86 super(name, Math.max(writeHandlers + readHandlers, numWriteQueues + numReadQueues), conf, abortable);
87
88 this.writeHandlersCount = Math.max(writeHandlers, numWriteQueues);
89 this.readHandlersCount = Math.max(readHandlers, numReadQueues);
90 this.numWriteQueues = numWriteQueues;
91 this.numReadQueues = numReadQueues;
92 this.writeBalancer = getBalancer(numWriteQueues);
93 this.readBalancer = getBalancer(numReadQueues);
94
95 queues = new ArrayList<BlockingQueue<CallRunner>>(writeHandlersCount + readHandlersCount);
96 LOG.debug(name + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount +
97 " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount);
98
99 for (int i = 0; i < numWriteQueues; ++i) {
100 queues.add((BlockingQueue<CallRunner>)
101 ReflectionUtils.newInstance(writeQueueClass, writeQueueInitArgs));
102 }
103
104 for (int i = 0; i < numReadQueues; ++i) {
105 queues.add((BlockingQueue<CallRunner>)
106 ReflectionUtils.newInstance(readQueueClass, readQueueInitArgs));
107 }
108 }
109
110 @Override
111 protected void startHandlers(final int port) {
112 startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port);
113 startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port);
114 }
115
116 @Override
117 public void dispatch(final CallRunner callTask) throws InterruptedException {
118 RpcServer.Call call = callTask.getCall();
119 int queueIndex;
120 if (isWriteRequest(call.getHeader(), call.param)) {
121 queueIndex = writeBalancer.getNextQueue();
122 } else {
123 queueIndex = numWriteQueues + readBalancer.getNextQueue();
124 }
125 queues.get(queueIndex).put(callTask);
126 }
127
128 private boolean isWriteRequest(final RequestHeader header, final Message param) {
129
130 String methodName = header.getMethodName();
131 if (methodName.equalsIgnoreCase("multi") && param instanceof MultiRequest) {
132 MultiRequest multi = (MultiRequest)param;
133 for (RegionAction regionAction : multi.getRegionActionList()) {
134 for (Action action: regionAction.getActionList()) {
135 if (action.hasMutation()) {
136 return true;
137 }
138 }
139 }
140 }
141 return false;
142 }
143
144 @Override
145 public int getQueueLength() {
146 int length = 0;
147 for (final BlockingQueue<CallRunner> queue: queues) {
148 length += queue.size();
149 }
150 return length;
151 }
152
153 @Override
154 protected List<BlockingQueue<CallRunner>> getQueues() {
155 return queues;
156 }
157
158
159
160
161
162 private static int calcNumWriters(final int count, final float readShare) {
163 return Math.max(1, count - Math.max(1, (int)Math.round(count * readShare)));
164 }
165
166
167
168
169
170 private static int calcNumReaders(final int count, final float readShare) {
171 return count - calcNumWriters(count, readShare);
172 }
173 }