1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
9 *
10 * http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
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.Semaphore;
23 import java.util.concurrent.atomic.AtomicInteger;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26
27 @InterfaceAudience.Private
28 public class FastPathRpcHandler extends RpcHandler {
29 // Below are for fast-path support. Push this Handler on to the fastPathHandlerStack Deque
30 // if an empty queue of CallRunners so we are available for direct handoff when one comes in.
31 final Deque<FastPathRpcHandler> fastPathHandlerStack;
32 // Semaphore to coordinate loading of fastpathed loadedTask and our running it.
33 // UNFAIR synchronization.
34 private Semaphore semaphore = new Semaphore(0);
35 // The task we get when fast-pathing.
36 private CallRunner loadedCallRunner;
37
38 FastPathRpcHandler(String name, double handlerFailureThreshhold, int handlerCount,
39 BlockingQueue<CallRunner> q, AtomicInteger activeHandlerCount,
40 AtomicInteger failedHandlerCount, final Abortable abortable,
41 final Deque<FastPathRpcHandler> fastPathHandlerStack) {
42 super(name, handlerFailureThreshhold, handlerCount, q, activeHandlerCount, failedHandlerCount,
43 abortable);
44 this.fastPathHandlerStack = fastPathHandlerStack;
45 }
46
47 @Override
48 protected CallRunner getCallRunner() throws InterruptedException {
49 // Get a callrunner if one in the Q.
50 CallRunner cr = this.q.poll();
51 if (cr == null) {
52 // Else, if a fastPathHandlerStack present and no callrunner in Q, register ourselves for
53 // the fastpath handoff done via fastPathHandlerStack.
54 if (this.fastPathHandlerStack != null) {
55 this.fastPathHandlerStack.push(this);
56 this.semaphore.acquire();
57 cr = this.loadedCallRunner;
58 this.loadedCallRunner = null;
59 } else {
60 // No fastpath available. Block until a task comes available.
61 cr = super.getCallRunner();
62 }
63 }
64 return cr;
65 }
66
67 /**
68 * @param cr Task gotten via fastpath.
69 * @return True if we successfully loaded our task
70 */
71 boolean loadCallRunner(final CallRunner cr) {
72 this.loadedCallRunner = cr;
73 this.semaphore.release();
74 return true;
75 }
76 }