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.DaemonThreadFactory;
24
25 import java.io.IOException;
26 import java.util.concurrent.ArrayBlockingQueue;
27 import java.util.concurrent.ThreadPoolExecutor;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicInteger;
30
31
32
33
34
35
36 public class FifoRpcScheduler extends RpcScheduler {
37 private static final Log LOG = LogFactory.getLog(FifoRpcScheduler.class);
38 private final int handlerCount;
39 private final int maxQueueLength;
40 private final AtomicInteger queueSize = new AtomicInteger(0);
41 private ThreadPoolExecutor executor;
42
43 public FifoRpcScheduler(Configuration conf, int handlerCount) {
44 this.handlerCount = handlerCount;
45 this.maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
46 handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
47 LOG.info("Using " + this.getClass().getSimpleName() + " as user call queue; handlerCount=" +
48 handlerCount + "; maxQueueLength=" + maxQueueLength);
49 }
50
51 @Override
52 public void init(Context context) {
53
54 }
55
56 @Override
57 public void start() {
58 this.executor = new ThreadPoolExecutor(
59 handlerCount,
60 handlerCount,
61 60,
62 TimeUnit.SECONDS,
63 new ArrayBlockingQueue<Runnable>(maxQueueLength),
64 new DaemonThreadFactory("FifoRpcScheduler.handler"),
65 new ThreadPoolExecutor.CallerRunsPolicy());
66 }
67
68 @Override
69 public void stop() {
70 this.executor.shutdown();
71 }
72
73 @Override
74 public boolean dispatch(final CallRunner task) throws IOException, InterruptedException {
75
76 int queued = queueSize.getAndIncrement();
77 if (maxQueueLength > 0 && queued >= maxQueueLength) {
78 queueSize.decrementAndGet();
79 return false;
80 }
81 executor.submit(new Runnable() {
82 @Override
83 public void run() {
84 task.setStatus(RpcServer.getStatus());
85 task.run();
86 queueSize.decrementAndGet();
87 }
88 });
89 return true;
90 }
91
92 @Override
93 public int getGeneralQueueLength() {
94 return executor.getQueue().size();
95 }
96
97 @Override
98 public int getPriorityQueueLength() {
99 return 0;
100 }
101
102 @Override
103 public int getReplicationQueueLength() {
104 return 0;
105 }
106
107 @Override
108 public int getActiveRpcHandlerCount() {
109 return executor.getActiveCount();
110 }
111
112 @Override
113 public int getActiveGeneralRpcHandlerCount() {
114 return getActiveRpcHandlerCount();
115 }
116
117 @Override
118 public int getActivePriorityRpcHandlerCount() {
119 return 0;
120 }
121
122 @Override
123 public int getActiveReplicationRpcHandlerCount() {
124 return 0;
125 }
126
127 @Override
128 public long getNumGeneralCallsDropped() {
129 return 0;
130 }
131
132 @Override
133 public long getNumLifoModeSwitches() {
134 return 0;
135 }
136
137 @Override
138 public int getWriteQueueLength() {
139 return 0;
140 }
141
142 @Override
143 public int getReadQueueLength() {
144 return 0;
145 }
146
147 @Override
148 public int getScanQueueLength() {
149 return 0;
150 }
151
152 @Override
153 public int getActiveWriteRpcHandlerCount() {
154 return 0;
155 }
156
157 @Override
158 public int getActiveReadRpcHandlerCount() {
159 return 0;
160 }
161
162 @Override
163 public int getActiveScanRpcHandlerCount() {
164 return 0;
165 }
166 }