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.Random;
24 import java.util.concurrent.BlockingQueue;
25 import java.util.concurrent.atomic.AtomicInteger;
26
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.HConstants;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.classification.InterfaceStability;
34 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
35 import org.apache.hadoop.util.StringUtils;
36
37 import com.google.common.base.Preconditions;
38 import com.google.common.base.Strings;
39
40 @InterfaceAudience.Private
41 @InterfaceStability.Evolving
42 public abstract class RpcExecutor {
43 private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
44
45 private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
46 private final List<Thread> handlers;
47 private final int handlerCount;
48 private final String name;
49 private final AtomicInteger failedHandlerCount = new AtomicInteger(0);
50
51 private boolean running;
52
53 private Configuration conf = null;
54 private Abortable abortable = null;
55
56 public RpcExecutor(final String name, final int handlerCount) {
57 this.handlers = new ArrayList<Thread>(handlerCount);
58 this.handlerCount = handlerCount;
59 this.name = Strings.nullToEmpty(name);
60 }
61
62 public RpcExecutor(final String name, final int handlerCount, final Configuration conf,
63 final Abortable abortable) {
64 this(name, handlerCount);
65 this.conf = conf;
66 this.abortable = abortable;
67 }
68
69 public void start(final int port) {
70 running = true;
71 startHandlers(port);
72 }
73
74 public void stop() {
75 running = false;
76 for (Thread handler : handlers) {
77 handler.interrupt();
78 }
79 }
80
81 public int getActiveHandlerCount() {
82 return activeHandlerCount.get();
83 }
84
85
86 public abstract int getQueueLength();
87
88
89 public abstract void dispatch(final CallRunner callTask) throws InterruptedException;
90
91
92 protected abstract List<BlockingQueue<CallRunner>> getQueues();
93
94 protected void startHandlers(final int port) {
95 List<BlockingQueue<CallRunner>> callQueues = getQueues();
96 startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port);
97 }
98
99 protected void startHandlers(final String nameSuffix, final int numHandlers,
100 final List<BlockingQueue<CallRunner>> callQueues,
101 final int qindex, final int qsize, final int port) {
102 final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
103 for (int i = 0; i < numHandlers; i++) {
104 final int index = qindex + (i % qsize);
105 Thread t = new Thread(new Runnable() {
106 @Override
107 public void run() {
108 consumerLoop(callQueues.get(index));
109 }
110 });
111 t.setDaemon(true);
112 t.setName(threadPrefix + "RpcServer.handler=" + handlers.size() +
113 ",queue=" + index + ",port=" + port);
114 t.start();
115 LOG.debug(threadPrefix + " Start Handler index=" + handlers.size() + " queue=" + index);
116 handlers.add(t);
117 }
118 }
119
120 protected void consumerLoop(final BlockingQueue<CallRunner> myQueue) {
121 boolean interrupted = false;
122 double handlerFailureThreshhold =
123 conf == null ? 1.0 : conf.getFloat(HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
124 HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
125 try {
126 while (running) {
127 try {
128 MonitoredRPCHandler status = RpcServer.getStatus();
129 CallRunner task = myQueue.take();
130 task.setStatus(status);
131 try {
132 activeHandlerCount.incrementAndGet();
133 task.run();
134 } catch (Error e) {
135 int failedCount = failedHandlerCount.incrementAndGet();
136 if (handlerFailureThreshhold >= 0
137 && failedCount > handlerCount * handlerFailureThreshhold) {
138 String message =
139 "Number of failed RpcServer handler exceeded threshhold "
140 + handlerFailureThreshhold + " with failed reason: "
141 + StringUtils.stringifyException(e);
142 if (abortable != null) {
143 abortable.abort(message, e);
144 } else {
145 LOG.error("Received " + StringUtils.stringifyException(e)
146 + " but not aborting due to abortable being null");
147 throw e;
148 }
149 } else {
150 LOG.warn("RpcServer handler threads encountered errors "
151 + StringUtils.stringifyException(e));
152 }
153 } finally {
154 activeHandlerCount.decrementAndGet();
155 }
156 } catch (InterruptedException e) {
157 interrupted = true;
158 }
159 }
160 } finally {
161 if (interrupted) {
162 Thread.currentThread().interrupt();
163 }
164 }
165 }
166
167 public static abstract class QueueBalancer {
168
169
170
171 public abstract int getNextQueue();
172 }
173
174 public static QueueBalancer getBalancer(int queueSize) {
175 Preconditions.checkArgument(queueSize > 0, "Queue size is <= 0, must be at least 1");
176 if (queueSize == 1) {
177 return ONE_QUEUE;
178 } else {
179 return new RandomQueueBalancer(queueSize);
180 }
181 }
182
183
184
185
186 private static QueueBalancer ONE_QUEUE = new QueueBalancer() {
187
188 @Override
189 public int getNextQueue() {
190 return 0;
191 }
192 };
193
194
195
196
197 private static class RandomQueueBalancer extends QueueBalancer {
198 private final int queueSize;
199
200 private final ThreadLocal<Random> threadRandom =
201 new ThreadLocal<Random>() {
202 @Override
203 protected Random initialValue() {
204 return new Random();
205 }
206 };
207
208 public RandomQueueBalancer(int queueSize) {
209 this.queueSize = queueSize;
210 }
211
212 public int getNextQueue() {
213 return threadRandom.get().nextInt(queueSize);
214 }
215 }
216 }