View Javadoc

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.ConcurrentLinkedDeque;
23  import java.util.concurrent.LinkedBlockingQueue;
24  import java.util.concurrent.Semaphore;
25  import java.util.concurrent.atomic.AtomicLong;
26  
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.hbase.Abortable;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  
31  /**
32   * Balanced queue executor with a fastpath. Because this is FIFO, it has no respect for
33   * ordering so a fast path skipping the queuing of Calls if an Handler is available, is possible.
34   * Just pass the Call direct to waiting Handler thread. Try to keep the hot Handlers bubbling
35   * rather than let them go cold and lose context. Idea taken from Apace Kudu (incubating). See
36   * https://gerrit.cloudera.org/#/c/2938/7/src/kudu/rpc/service_queue.h
37   */
38  @InterfaceAudience.Private
39  public class FastPathBalancedQueueRpcExecutor extends BalancedQueueRpcExecutor {
40    // Depends on default behavior of BalancedQueueRpcExecutor being FIFO!
41  
42    /*
43     * Stack of Handlers waiting for work.
44     */
45    private final Deque<FastPathHandler> fastPathHandlerStack = new ConcurrentLinkedDeque<>();
46  
47    public FastPathBalancedQueueRpcExecutor(final String name, final int handlerCount,
48                                            final int numQueues, final int maxQueueLength, final Configuration conf,
49                                            final Abortable abortable) {
50      super(name, handlerCount, numQueues, conf, abortable, LinkedBlockingQueue.class,
51          maxQueueLength);
52    }
53  
54    public FastPathBalancedQueueRpcExecutor(String name, int handlerCount,
55                                            int numCallQueues,
56                                            Configuration conf,
57                                            Abortable abortable,
58                                            Class<? extends BlockingQueue> queueClass,
59                                            Object... args) {
60      super(name, handlerCount, numCallQueues, conf, abortable, queueClass, args);
61    }
62  
63    @Override
64    protected Handler getHandler(String name, double handlerFailureThreshhold,
65        BlockingQueue<CallRunner> q) {
66      return new FastPathHandler(name, handlerFailureThreshhold, q, fastPathHandlerStack);
67    }
68  
69    @Override
70    public boolean dispatch(CallRunner callTask) throws InterruptedException {
71      FastPathHandler handler = popReadyHandler();
72      return handler != null? handler.loadCallRunner(callTask): super.dispatch(callTask);
73    }
74  
75    /**
76     * @return Pop a Handler instance if one available ready-to-go or else return null.
77     */
78    private FastPathHandler popReadyHandler() {
79      return this.fastPathHandlerStack.poll();
80    }
81  
82    class FastPathHandler extends Handler {
83      // Below are for fast-path support. Push this Handler on to the fastPathHandlerStack Deque
84      // if an empty queue of CallRunners so we are available for direct handoff when one comes in.
85      final Deque<FastPathHandler> fastPathHandlerStack;
86      // Semaphore to coordinate loading of fastpathed loadedTask and our running it.
87      private Semaphore semaphore = new Semaphore(0);
88      // The task we get when fast-pathing.
89      private CallRunner loadedCallRunner;
90  
91      FastPathHandler(String name, double handlerFailureThreshhold, BlockingQueue<CallRunner> q,
92          final Deque<FastPathHandler> fastPathHandlerStack) {
93        super(name, handlerFailureThreshhold, q);
94        this.fastPathHandlerStack = fastPathHandlerStack;
95      }
96  
97      protected CallRunner getCallRunner() throws InterruptedException {
98        // Get a callrunner if one in the Q.
99        CallRunner cr = this.q.poll();
100       if (cr == null) {
101         // Else, if a fastPathHandlerStack present and no callrunner in Q, register ourselves for
102         // the fastpath handoff done via fastPathHandlerStack.
103         if (this.fastPathHandlerStack != null) {
104           this.fastPathHandlerStack.push(this);
105           this.semaphore.acquire();
106           cr = this.loadedCallRunner;
107           this.loadedCallRunner = null;
108         } else {
109           // No fastpath available. Block until a task comes available.
110           cr = super.getCallRunner();
111         }
112       }
113       return cr;
114     }
115 
116     /**
117      * @param task Task gotten via fastpath.
118      * @return True if we successfully loaded our task
119      */
120     boolean loadCallRunner(final CallRunner cr) {
121       this.loadedCallRunner = cr;
122       this.semaphore.release();
123       return true;
124     }
125   }
126 }