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.hadoop.conf.Configuration;
21 import org.apache.hadoop.hbase.DaemonThreadFactory;
22
23 import java.io.IOException;
24 import java.util.concurrent.ArrayBlockingQueue;
25 import java.util.concurrent.ThreadPoolExecutor;
26 import java.util.concurrent.TimeUnit;
27 import org.apache.hadoop.hbase.ipc.CallRunner;
28
29
30
31
32
33
34 public class FifoRpcScheduler extends RpcScheduler {
35
36 private final int handlerCount;
37 private final int maxQueueLength;
38 private ThreadPoolExecutor executor;
39
40 public FifoRpcScheduler(Configuration conf, int handlerCount) {
41 this.handlerCount = handlerCount;
42 this.maxQueueLength = conf.getInt("hbase.ipc.server.max.callqueue.length",
43 handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
44 }
45
46 @Override
47 public void init(Context context) {
48
49 }
50
51 @Override
52 public void start() {
53 this.executor = new ThreadPoolExecutor(
54 handlerCount,
55 handlerCount,
56 60,
57 TimeUnit.SECONDS,
58 new ArrayBlockingQueue<Runnable>(maxQueueLength),
59 new DaemonThreadFactory("FifoRpcScheduler.handler"),
60 new ThreadPoolExecutor.CallerRunsPolicy());
61 }
62
63 @Override
64 public void stop() {
65 this.executor.shutdown();
66 }
67
68 @Override
69 public void dispatch(final CallRunner task) throws IOException, InterruptedException {
70 executor.submit(new Runnable() {
71 @Override
72 public void run() {
73 task.run();
74 }
75 });
76 }
77
78 @Override
79 public int getGeneralQueueLength() {
80 return executor.getQueue().size();
81 }
82
83 @Override
84 public int getPriorityQueueLength() {
85 return 0;
86 }
87
88 @Override
89 public int getReplicationQueueLength() {
90 return 0;
91 }
92
93 @Override
94 public int getActiveRpcHandlerCount() {
95 return executor.getActiveCount();
96 }
97 }