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}