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  
19  package org.apache.hadoop.hbase.ipc;
20  
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.concurrent.BlockingQueue;
24  import java.util.concurrent.atomic.AtomicInteger;
25  import java.util.Random;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.classification.InterfaceStability;
31  
32  import com.google.common.base.Preconditions;
33  import com.google.common.base.Strings;
34  
35  @InterfaceAudience.Private
36  @InterfaceStability.Evolving
37  public abstract class RpcExecutor {
38    private static final Log LOG = LogFactory.getLog(RpcExecutor.class);
39  
40    private final AtomicInteger activeHandlerCount = new AtomicInteger(0);
41    private final List<Thread> handlers;
42    private final int handlerCount;
43    private final String name;
44  
45    private boolean running;
46  
47    public RpcExecutor(final String name, final int handlerCount) {
48      this.handlers = new ArrayList<Thread>(handlerCount);
49      this.handlerCount = handlerCount;
50      this.name = Strings.nullToEmpty(name);
51    }
52  
53    public void start(final int port) {
54      running = true;
55      startHandlers(port);
56    }
57  
58    public void stop() {
59      running = false;
60      for (Thread handler : handlers) {
61        handler.interrupt();
62      }
63    }
64  
65    public int getActiveHandlerCount() {
66      return activeHandlerCount.get();
67    }
68  
69    /** Returns the length of the pending queue */
70    public abstract int getQueueLength();
71  
72    /** Add the request to the executor queue */
73    public abstract void dispatch(final CallRunner callTask) throws InterruptedException;
74  
75    /** Returns the list of request queues */
76    protected abstract List<BlockingQueue<CallRunner>> getQueues();
77  
78    protected void startHandlers(final int port) {
79      List<BlockingQueue<CallRunner>> callQueues = getQueues();
80      startHandlers(null, handlerCount, callQueues, 0, callQueues.size(), port);
81    }
82  
83    protected void startHandlers(final String nameSuffix, final int numHandlers,
84        final List<BlockingQueue<CallRunner>> callQueues,
85        final int qindex, final int qsize, final int port) {
86      final String threadPrefix = name + Strings.nullToEmpty(nameSuffix);
87      for (int i = 0; i < numHandlers; i++) {
88        final int index = qindex + (i % qsize);
89        Thread t = new Thread(new Runnable() {
90          @Override
91          public void run() {
92            consumerLoop(callQueues.get(index));
93          }
94        });
95        t.setDaemon(true);
96        t.setName(threadPrefix + "RpcServer.handler=" + handlers.size() +
97            ",queue=" + index + ",port=" + port);
98        t.start();
99        LOG.debug(threadPrefix + " Start Handler index=" + handlers.size() + " queue=" + index);
100       handlers.add(t);
101     }
102   }
103 
104   protected void consumerLoop(final BlockingQueue<CallRunner> myQueue) {
105     boolean interrupted = false;
106     try {
107       while (running) {
108         try {
109           CallRunner task = myQueue.take();
110           try {
111             activeHandlerCount.incrementAndGet();
112             task.run();
113           } catch (Error e) {
114             LOG.error("RpcServer handler thread throws error: ", e);
115             throw e;
116           } catch (RuntimeException e) {
117             LOG.error("RpcServer handler thread throws exception: ", e);
118             throw e;
119           } finally {
120             activeHandlerCount.decrementAndGet();
121           }
122         } catch (InterruptedException e) {
123           interrupted = true;
124         }
125       }
126     } finally {
127       if (interrupted) {
128         Thread.currentThread().interrupt();
129       }
130     }
131   }
132 
133   public static abstract class QueueBalancer {
134     /**
135      * @return the index of the next queue to which a request should be inserted
136      */
137     public abstract int getNextQueue();
138   }
139 
140   public static QueueBalancer getBalancer(int queueSize) {
141     Preconditions.checkArgument(queueSize > 0, "Queue size is <= 0, must be at least 1");
142     if (queueSize == 1) {
143       return ONE_QUEUE;
144     } else {
145       return new RandomQueueBalancer(queueSize);
146     }
147   }
148 
149   /**
150    * All requests go to the first queue, at index 0
151    */
152   private static QueueBalancer ONE_QUEUE = new QueueBalancer() {
153 
154     @Override
155     public int getNextQueue() {
156       return 0;
157     }
158   };
159 
160   /**
161    * Queue balancer that just randomly selects a queue in the range [0, num queues).
162    */
163   private static class RandomQueueBalancer extends QueueBalancer {
164     private final int queueSize;
165 
166     private final ThreadLocal<Random> threadRandom =
167       new ThreadLocal<Random>() {
168         @Override
169         protected Random initialValue() {
170           return new Random();
171         }
172       };
173 
174     public RandomQueueBalancer(int queueSize) {
175       this.queueSize = queueSize;
176     }
177 
178     public int getNextQueue() {
179       return threadRandom.get().nextInt(queueSize);
180     }
181   }
182 }