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