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 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   * A very simple {@code }RpcScheduler} that serves incoming requests in order.
31   *
32   * This can be used for HMaster, where no prioritization is needed.
33   */
34  public class FifoRpcScheduler implements 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("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      // no-op
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  }