001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.ipc;
019
020import java.io.IOException;
021import java.util.HashMap;
022import java.util.concurrent.ArrayBlockingQueue;
023import java.util.concurrent.ThreadPoolExecutor;
024import java.util.concurrent.TimeUnit;
025import java.util.concurrent.atomic.AtomicInteger;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.util.Threads;
028import org.apache.yetus.audience.InterfaceAudience;
029import org.apache.yetus.audience.InterfaceStability;
030import org.slf4j.Logger;
031import org.slf4j.LoggerFactory;
032
033import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
034
035/**
036 * A special {@code }RpcScheduler} only used for master. This scheduler separates RegionServerReport
037 * requests to independent handlers to avoid these requests block other requests. To use this
038 * scheduler, please set "hbase.master.rpc.scheduler.factory.class" to
039 * "org.apache.hadoop.hbase.ipc.MasterFifoRpcScheduler".
040 */
041@InterfaceAudience.Private
042@InterfaceStability.Evolving
043public class MasterFifoRpcScheduler extends FifoRpcScheduler {
044  private static final Logger LOG = LoggerFactory.getLogger(MasterFifoRpcScheduler.class);
045
046  /**
047   * Set RSReport requests handlers count when masters use MasterFifoRpcScheduler. The default value
048   * is half of "hbase.regionserver.handler.count" value, but at least 1. The other handlers count
049   * is "hbase.regionserver.handler.count" value minus RSReport handlers count, but at least 1 too.
050   */
051  public static final String MASTER_SERVER_REPORT_HANDLER_COUNT =
052      "hbase.master.server.report.handler.count";
053  private static final String REGION_SERVER_REPORT = "RegionServerReport";
054  private final int rsReportHandlerCount;
055  private final int rsRsreportMaxQueueLength;
056  private final AtomicInteger rsReportQueueSize = new AtomicInteger(0);
057  private ThreadPoolExecutor rsReportExecutor;
058
059  public MasterFifoRpcScheduler(Configuration conf, int callHandlerCount,
060      int rsReportHandlerCount) {
061    super(conf, callHandlerCount);
062    this.rsReportHandlerCount = rsReportHandlerCount;
063    this.rsRsreportMaxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
064      rsReportHandlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
065  }
066
067  @Override
068  public void start() {
069    LOG.info(
070      "Using {} as call queue; handlerCount={}; maxQueueLength={}; rsReportHandlerCount={}; "
071          + "rsReportMaxQueueLength={}",
072      this.getClass().getSimpleName(), handlerCount, maxQueueLength, rsReportHandlerCount,
073      rsRsreportMaxQueueLength);
074    this.executor = new ThreadPoolExecutor(handlerCount, handlerCount, 60, TimeUnit.SECONDS,
075      new ArrayBlockingQueue<>(maxQueueLength),
076      new ThreadFactoryBuilder().setNameFormat("MasterFifoRpcScheduler.call.handler-pool-%d")
077        .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
078      new ThreadPoolExecutor.CallerRunsPolicy());
079    this.rsReportExecutor = new ThreadPoolExecutor(rsReportHandlerCount, rsReportHandlerCount, 60,
080      TimeUnit.SECONDS, new ArrayBlockingQueue<>(rsRsreportMaxQueueLength),
081      new ThreadFactoryBuilder().setNameFormat("MasterFifoRpcScheduler.RSReport.handler-pool-%d")
082        .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
083      new ThreadPoolExecutor.CallerRunsPolicy());
084  }
085
086  @Override
087  public void stop() {
088    this.executor.shutdown();
089    this.rsReportExecutor.shutdown();
090  }
091
092  @Override
093  public boolean dispatch(final CallRunner task) throws IOException, InterruptedException {
094    String method = getCallMethod(task);
095    if (rsReportExecutor != null && method != null && method.equals(REGION_SERVER_REPORT)) {
096      return executeRpcCall(rsReportExecutor, rsReportQueueSize, task);
097    } else {
098      return executeRpcCall(executor, queueSize, task);
099    }
100  }
101
102  @Override
103  public int getGeneralQueueLength() {
104    return executor.getQueue().size() + rsReportExecutor.getQueue().size();
105  }
106
107  @Override
108  public int getActiveRpcHandlerCount() {
109    return executor.getActiveCount() + rsReportExecutor.getActiveCount();
110  }
111
112  @Override
113  public CallQueueInfo getCallQueueInfo() {
114    String queueName = "Master Fifo Queue";
115
116    HashMap<String, Long> methodCount = new HashMap<>();
117    HashMap<String, Long> methodSize = new HashMap<>();
118
119    CallQueueInfo callQueueInfo = new CallQueueInfo();
120    callQueueInfo.setCallMethodCount(queueName, methodCount);
121    callQueueInfo.setCallMethodSize(queueName, methodSize);
122
123    updateMethodCountAndSizeByQueue(executor.getQueue(), methodCount, methodSize);
124    updateMethodCountAndSizeByQueue(rsReportExecutor.getQueue(), methodCount, methodSize);
125
126    return callQueueInfo;
127  }
128}