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 java.util.Deque;
21 import java.util.concurrent.BlockingQueue;
22 import java.util.concurrent.ConcurrentLinkedDeque;
23 import java.util.concurrent.Semaphore;
24 import java.util.concurrent.atomic.AtomicInteger;
25
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.Abortable;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29
30
31
32
33
34
35
36
37 @InterfaceAudience.Private
38 public class FastPathBalancedQueueRpcExecutor extends BalancedQueueRpcExecutor {
39
40
41
42
43
44 private final Deque<FastPathHandler> fastPathHandlerStack = new ConcurrentLinkedDeque<>();
45
46 public FastPathBalancedQueueRpcExecutor(final String name, final int handlerCount,
47 final int maxQueueLength, final PriorityFunction priority, final Configuration conf,
48 final Abortable abortable) {
49 super(name, handlerCount, maxQueueLength, priority, conf, abortable);
50
51 }
52
53 public FastPathBalancedQueueRpcExecutor(final String name, final int handlerCount,
54 final String callQueueType, final int maxQueueLength, final PriorityFunction priority,
55 final Configuration conf, final Abortable abortable) {
56 super(name, handlerCount, callQueueType, maxQueueLength, priority, conf, abortable);
57 }
58
59 @Override
60 protected Handler getHandler(String name, double handlerFailureThreshhold,
61 BlockingQueue<CallRunner> q, AtomicInteger activeHandlerCount) {
62 return new FastPathHandler(name, handlerFailureThreshhold, q, activeHandlerCount,
63 fastPathHandlerStack);
64 }
65
66 @Override
67 public boolean dispatch(CallRunner callTask) throws InterruptedException {
68
69
70 if (currentQueueLimit == 0){
71 return false;
72 }
73 FastPathHandler handler = popReadyHandler();
74 return handler != null? handler.loadCallRunner(callTask): super.dispatch(callTask);
75 }
76
77
78
79
80 private FastPathHandler popReadyHandler() {
81 return this.fastPathHandlerStack.poll();
82 }
83
84 class FastPathHandler extends Handler {
85
86
87 final Deque<FastPathHandler> fastPathHandlerStack;
88
89 private Semaphore semaphore = new Semaphore(0);
90
91 private CallRunner loadedCallRunner;
92
93 FastPathHandler(String name, double handlerFailureThreshhold, BlockingQueue<CallRunner> q,
94 final AtomicInteger activeHandlerCount,
95 final Deque<FastPathHandler> fastPathHandlerStack) {
96 super(name, handlerFailureThreshhold, q, activeHandlerCount);
97 this.fastPathHandlerStack = fastPathHandlerStack;
98 }
99
100 protected CallRunner getCallRunner() throws InterruptedException {
101
102 CallRunner cr = this.q.poll();
103 if (cr == null) {
104
105
106 if (this.fastPathHandlerStack != null) {
107 this.fastPathHandlerStack.push(this);
108 this.semaphore.acquire();
109 cr = this.loadedCallRunner;
110 this.loadedCallRunner = null;
111 } else {
112
113 cr = super.getCallRunner();
114 }
115 }
116 return cr;
117 }
118
119
120
121
122
123 boolean loadCallRunner(final CallRunner cr) {
124 this.loadedCallRunner = cr;
125 this.semaphore.release();
126 return true;
127 }
128 }
129 }