001/** 002 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019 020package org.apache.hadoop.hbase.ipc; 021 022import java.util.concurrent.BlockingQueue; 023import java.util.concurrent.atomic.AtomicInteger; 024 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.Abortable; 027import org.apache.hadoop.hbase.HBaseInterfaceAudience; 028import org.apache.yetus.audience.InterfaceAudience; 029import org.apache.yetus.audience.InterfaceStability; 030import org.slf4j.Logger; 031import org.slf4j.LoggerFactory; 032import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Action; 033import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; 034import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; 035import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 036import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos; 037import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; 038import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; 039import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; 040import org.apache.hbase.thirdparty.com.google.protobuf.Message; 041 042/** 043 * RPC Executor that uses different queues for reads and writes. 044 * With the options to use different queues/executors for gets and scans. 045 * Each handler has its own queue and there is no stealing. 046 */ 047@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) 048@InterfaceStability.Evolving 049public class RWQueueRpcExecutor extends RpcExecutor { 050 private static final Logger LOG = LoggerFactory.getLogger(RWQueueRpcExecutor.class); 051 052 public static final String CALL_QUEUE_READ_SHARE_CONF_KEY = 053 "hbase.ipc.server.callqueue.read.ratio"; 054 public static final String CALL_QUEUE_SCAN_SHARE_CONF_KEY = 055 "hbase.ipc.server.callqueue.scan.ratio"; 056 057 private final QueueBalancer writeBalancer; 058 private final QueueBalancer readBalancer; 059 private final QueueBalancer scanBalancer; 060 private final int writeHandlersCount; 061 private final int readHandlersCount; 062 private final int scanHandlersCount; 063 private final int numWriteQueues; 064 private final int numReadQueues; 065 private final int numScanQueues; 066 067 private final AtomicInteger activeWriteHandlerCount = new AtomicInteger(0); 068 private final AtomicInteger activeReadHandlerCount = new AtomicInteger(0); 069 private final AtomicInteger activeScanHandlerCount = new AtomicInteger(0); 070 071 public RWQueueRpcExecutor(final String name, final int handlerCount, final int maxQueueLength, 072 final PriorityFunction priority, final Configuration conf, final Abortable abortable) { 073 super(name, handlerCount, maxQueueLength, priority, conf, abortable); 074 075 float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0); 076 float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0); 077 078 numWriteQueues = calcNumWriters(this.numCallQueues, callqReadShare); 079 writeHandlersCount = Math.max(numWriteQueues, calcNumWriters(handlerCount, callqReadShare)); 080 081 int readQueues = calcNumReaders(this.numCallQueues, callqReadShare); 082 int readHandlers = Math.max(readQueues, calcNumReaders(handlerCount, callqReadShare)); 083 084 int scanQueues = Math.max(0, (int)Math.floor(readQueues * callqScanShare)); 085 int scanHandlers = Math.max(0, (int)Math.floor(readHandlers * callqScanShare)); 086 087 if ((readQueues - scanQueues) > 0) { 088 readQueues -= scanQueues; 089 readHandlers -= scanHandlers; 090 } else { 091 scanQueues = 0; 092 scanHandlers = 0; 093 } 094 095 numReadQueues = readQueues; 096 readHandlersCount = readHandlers; 097 numScanQueues = scanQueues; 098 scanHandlersCount = scanHandlers; 099 100 this.writeBalancer = getBalancer(numWriteQueues); 101 this.readBalancer = getBalancer(numReadQueues); 102 this.scanBalancer = numScanQueues > 0 ? getBalancer(numScanQueues) : null; 103 104 initializeQueues(numWriteQueues); 105 initializeQueues(numReadQueues); 106 initializeQueues(numScanQueues); 107 108 LOG.info(getName() + " writeQueues=" + numWriteQueues + " writeHandlers=" + writeHandlersCount 109 + " readQueues=" + numReadQueues + " readHandlers=" + readHandlersCount + " scanQueues=" 110 + numScanQueues + " scanHandlers=" + scanHandlersCount); 111 } 112 113 @Override 114 protected int computeNumCallQueues(final int handlerCount, final float callQueuesHandlersFactor) { 115 // at least 1 read queue and 1 write queue 116 return Math.max(2, (int) Math.round(handlerCount * callQueuesHandlersFactor)); 117 } 118 119 @Override 120 protected void startHandlers(final int port) { 121 startHandlers(".write", writeHandlersCount, queues, 0, numWriteQueues, port, 122 activeWriteHandlerCount); 123 startHandlers(".read", readHandlersCount, queues, numWriteQueues, numReadQueues, port, 124 activeReadHandlerCount); 125 if (numScanQueues > 0) { 126 startHandlers(".scan", scanHandlersCount, queues, numWriteQueues + numReadQueues, 127 numScanQueues, port, activeScanHandlerCount); 128 } 129 } 130 131 @Override 132 public boolean dispatch(final CallRunner callTask) throws InterruptedException { 133 RpcCall call = callTask.getRpcCall(); 134 int queueIndex; 135 if (isWriteRequest(call.getHeader(), call.getParam())) { 136 queueIndex = writeBalancer.getNextQueue(); 137 } else if (numScanQueues > 0 && isScanRequest(call.getHeader(), call.getParam())) { 138 queueIndex = numWriteQueues + numReadQueues + scanBalancer.getNextQueue(); 139 } else { 140 queueIndex = numWriteQueues + readBalancer.getNextQueue(); 141 } 142 143 BlockingQueue<CallRunner> queue = queues.get(queueIndex); 144 if (queue.size() >= currentQueueLimit) { 145 return false; 146 } 147 return queue.offer(callTask); 148 } 149 150 @Override 151 public int getWriteQueueLength() { 152 int length = 0; 153 for (int i = 0; i < numWriteQueues; i++) { 154 length += queues.get(i).size(); 155 } 156 return length; 157 } 158 159 @Override 160 public int getReadQueueLength() { 161 int length = 0; 162 for (int i = numWriteQueues; i < (numWriteQueues + numReadQueues); i++) { 163 length += queues.get(i).size(); 164 } 165 return length; 166 } 167 168 @Override 169 public int getScanQueueLength() { 170 int length = 0; 171 for (int i = numWriteQueues + numReadQueues; 172 i < (numWriteQueues + numReadQueues + numScanQueues); i++) { 173 length += queues.get(i).size(); 174 } 175 return length; 176 } 177 178 @Override 179 public int getActiveHandlerCount() { 180 return activeWriteHandlerCount.get() + activeReadHandlerCount.get() 181 + activeScanHandlerCount.get(); 182 } 183 184 @Override 185 public int getActiveWriteHandlerCount() { 186 return activeWriteHandlerCount.get(); 187 } 188 189 @Override 190 public int getActiveReadHandlerCount() { 191 return activeReadHandlerCount.get(); 192 } 193 194 @Override 195 public int getActiveScanHandlerCount() { 196 return activeScanHandlerCount.get(); 197 } 198 199 private boolean isWriteRequest(final RequestHeader header, final Message param) { 200 // TODO: Is there a better way to do this? 201 if (param instanceof MultiRequest) { 202 MultiRequest multi = (MultiRequest)param; 203 for (RegionAction regionAction : multi.getRegionActionList()) { 204 for (Action action: regionAction.getActionList()) { 205 if (action.hasMutation()) { 206 return true; 207 } 208 } 209 } 210 } 211 if (param instanceof MutateRequest) { 212 return true; 213 } 214 // Below here are methods for master. It's a pretty brittle version of this. 215 // Not sure that master actually needs a read/write queue since 90% of requests to 216 // master are writing to status or changing the meta table. 217 // All other read requests are admin generated and can be processed whenever. 218 // However changing that would require a pretty drastic change and should be done for 219 // the next major release and not as a fix for HBASE-14239 220 if (param instanceof RegionServerStatusProtos.ReportRegionStateTransitionRequest) { 221 return true; 222 } 223 if (param instanceof RegionServerStatusProtos.RegionServerStartupRequest) { 224 return true; 225 } 226 if (param instanceof RegionServerStatusProtos.RegionServerReportRequest) { 227 return true; 228 } 229 return false; 230 } 231 232 private boolean isScanRequest(final RequestHeader header, final Message param) { 233 if (param instanceof ScanRequest) { 234 // The first scan request will be executed as a "short read" 235 ScanRequest request = (ScanRequest)param; 236 return request.hasScannerId(); 237 } 238 return false; 239 } 240 241 /* 242 * Calculate the number of writers based on the "total count" and the read share. 243 * You'll get at least one writer. 244 */ 245 private static int calcNumWriters(final int count, final float readShare) { 246 return Math.max(1, count - Math.max(1, (int)Math.round(count * readShare))); 247 } 248 249 /* 250 * Calculate the number of readers based on the "total count" and the read share. 251 * You'll get at least one reader. 252 */ 253 private static int calcNumReaders(final int count, final float readShare) { 254 return count - calcNumWriters(count, readShare); 255 } 256}