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.util.HashMap; 021import java.util.concurrent.ArrayBlockingQueue; 022import java.util.concurrent.ThreadPoolExecutor; 023import java.util.concurrent.TimeUnit; 024import java.util.concurrent.atomic.AtomicInteger; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.util.Threads; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.apache.yetus.audience.InterfaceStability; 029import org.slf4j.Logger; 030import org.slf4j.LoggerFactory; 031 032import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 033 034/** 035 * A special {@code }RpcScheduler} only used for master. This scheduler separates RegionServerReport 036 * requests to independent handlers to avoid these requests block other requests. To use this 037 * scheduler, please set "hbase.master.rpc.scheduler.factory.class" to 038 * "org.apache.hadoop.hbase.ipc.MasterFifoRpcScheduler". 039 */ 040@InterfaceAudience.Private 041@InterfaceStability.Evolving 042public class MasterFifoRpcScheduler extends FifoRpcScheduler { 043 private static final Logger LOG = LoggerFactory.getLogger(MasterFifoRpcScheduler.class); 044 045 /** 046 * Set RSReport requests handlers count when masters use MasterFifoRpcScheduler. The default value 047 * is half of "hbase.regionserver.handler.count" value, but at least 1. The other handlers count 048 * is "hbase.regionserver.handler.count" value minus RSReport handlers count, but at least 1 too. 049 */ 050 public static final String MASTER_SERVER_REPORT_HANDLER_COUNT = 051 "hbase.master.server.report.handler.count"; 052 private static final String REGION_SERVER_REPORT = "RegionServerReport"; 053 private final int rsReportHandlerCount; 054 private final int rsRsreportMaxQueueLength; 055 private final AtomicInteger rsReportQueueSize = new AtomicInteger(0); 056 private ThreadPoolExecutor rsReportExecutor; 057 058 public MasterFifoRpcScheduler(Configuration conf, int callHandlerCount, 059 int rsReportHandlerCount) { 060 super(conf, callHandlerCount); 061 this.rsReportHandlerCount = rsReportHandlerCount; 062 this.rsRsreportMaxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 063 rsReportHandlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); 064 } 065 066 @Override 067 public void start() { 068 LOG.info( 069 "Using {} as call queue; handlerCount={}; maxQueueLength={}; rsReportHandlerCount={}; " 070 + "rsReportMaxQueueLength={}", 071 this.getClass().getSimpleName(), handlerCount, maxQueueLength, rsReportHandlerCount, 072 rsRsreportMaxQueueLength); 073 this.executor = new ThreadPoolExecutor(handlerCount, handlerCount, 60, TimeUnit.SECONDS, 074 new ArrayBlockingQueue<>(maxQueueLength), 075 new ThreadFactoryBuilder().setNameFormat("MasterFifoRpcScheduler.call.handler-pool-%d") 076 .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(), 077 new ThreadPoolExecutor.CallerRunsPolicy()); 078 this.rsReportExecutor = new ThreadPoolExecutor(rsReportHandlerCount, rsReportHandlerCount, 60, 079 TimeUnit.SECONDS, new ArrayBlockingQueue<>(rsRsreportMaxQueueLength), 080 new ThreadFactoryBuilder().setNameFormat("MasterFifoRpcScheduler.RSReport.handler-pool-%d") 081 .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(), 082 new ThreadPoolExecutor.CallerRunsPolicy()); 083 } 084 085 @Override 086 public void stop() { 087 this.executor.shutdown(); 088 this.rsReportExecutor.shutdown(); 089 } 090 091 @Override 092 public boolean dispatch(final CallRunner task) { 093 String method = getCallMethod(task); 094 if (rsReportExecutor != null && method != null && method.equals(REGION_SERVER_REPORT)) { 095 return executeRpcCall(rsReportExecutor, rsReportQueueSize, task); 096 } else { 097 return executeRpcCall(executor, queueSize, task); 098 } 099 } 100 101 @Override 102 public int getGeneralQueueLength() { 103 return executor.getQueue().size() + rsReportExecutor.getQueue().size(); 104 } 105 106 @Override 107 public int getActiveRpcHandlerCount() { 108 return executor.getActiveCount() + rsReportExecutor.getActiveCount(); 109 } 110 111 @Override 112 public CallQueueInfo getCallQueueInfo() { 113 String queueName = "Master Fifo Queue"; 114 115 HashMap<String, Long> methodCount = new HashMap<>(); 116 HashMap<String, Long> methodSize = new HashMap<>(); 117 118 CallQueueInfo callQueueInfo = new CallQueueInfo(); 119 callQueueInfo.setCallMethodCount(queueName, methodCount); 120 callQueueInfo.setCallMethodSize(queueName, methodSize); 121 122 updateMethodCountAndSizeByQueue(executor.getQueue(), methodCount, methodSize); 123 updateMethodCountAndSizeByQueue(rsReportExecutor.getQueue(), methodCount, methodSize); 124 125 return callQueueInfo; 126 } 127}