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 */ 018 019package org.apache.hadoop.hbase.ipc; 020 021import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; 022 023import java.io.IOException; 024import java.net.InetAddress; 025import java.net.InetSocketAddress; 026import java.nio.ByteBuffer; 027import java.nio.channels.ReadableByteChannel; 028import java.nio.channels.WritableByteChannel; 029import java.util.Collections; 030import java.util.HashMap; 031import java.util.List; 032import java.util.Locale; 033import java.util.Map; 034import java.util.Optional; 035import java.util.concurrent.atomic.LongAdder; 036 037import org.apache.commons.lang3.StringUtils; 038import org.apache.hadoop.conf.Configuration; 039import org.apache.hadoop.hbase.CallQueueTooBigException; 040import org.apache.hadoop.hbase.CellScanner; 041import org.apache.hadoop.hbase.DoNotRetryIOException; 042import org.apache.hadoop.hbase.Server; 043import org.apache.hadoop.hbase.conf.ConfigurationObserver; 044import org.apache.hadoop.hbase.exceptions.RequestTooBigException; 045import org.apache.hadoop.hbase.io.ByteBuffAllocator; 046import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; 047import org.apache.hadoop.hbase.monitoring.TaskMonitor; 048import org.apache.hadoop.hbase.regionserver.RSRpcServices; 049import org.apache.hadoop.hbase.regionserver.slowlog.RpcLogDetails; 050import org.apache.hadoop.hbase.regionserver.slowlog.SlowLogRecorder; 051import org.apache.hadoop.hbase.security.HBasePolicyProvider; 052import org.apache.hadoop.hbase.security.SaslUtil; 053import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; 054import org.apache.hadoop.hbase.security.User; 055import org.apache.hadoop.hbase.security.UserProvider; 056import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager; 057import org.apache.hadoop.hbase.util.GsonUtil; 058import org.apache.hadoop.hbase.util.Pair; 059import org.apache.hadoop.security.UserGroupInformation; 060import org.apache.hadoop.security.authorize.AuthorizationException; 061import org.apache.hadoop.security.authorize.PolicyProvider; 062import org.apache.hadoop.security.authorize.ProxyUsers; 063import org.apache.hadoop.security.authorize.ServiceAuthorizationManager; 064import org.apache.hadoop.security.token.SecretManager; 065import org.apache.hadoop.security.token.TokenIdentifier; 066import org.apache.yetus.audience.InterfaceAudience; 067import org.slf4j.Logger; 068import org.slf4j.LoggerFactory; 069 070import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 071import org.apache.hbase.thirdparty.com.google.gson.Gson; 072import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; 073import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; 074import org.apache.hbase.thirdparty.com.google.protobuf.Message; 075import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 076import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; 077 078import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 079import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 080import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; 081 082/** 083 * An RPC server that hosts protobuf described Services. 084 * 085 */ 086@InterfaceAudience.Private 087public abstract class RpcServer implements RpcServerInterface, 088 ConfigurationObserver { 089 // LOG is being used in CallRunner and the log level is being changed in tests 090 public static final Logger LOG = LoggerFactory.getLogger(RpcServer.class); 091 protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION 092 = new CallQueueTooBigException(); 093 094 private static final String MULTI_GETS = "multi.gets"; 095 private static final String MULTI_MUTATIONS = "multi.mutations"; 096 private static final String MULTI_SERVICE_CALLS = "multi.service_calls"; 097 098 private final boolean authorize; 099 protected boolean isSecurityEnabled; 100 101 public static final byte CURRENT_VERSION = 0; 102 103 /** 104 * Whether we allow a fallback to SIMPLE auth for insecure clients when security is enabled. 105 */ 106 public static final String FALLBACK_TO_INSECURE_CLIENT_AUTH = 107 "hbase.ipc.server.fallback-to-simple-auth-allowed"; 108 109 /** 110 * How many calls/handler are allowed in the queue. 111 */ 112 protected static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10; 113 114 protected final CellBlockBuilder cellBlockBuilder; 115 116 protected static final String AUTH_FAILED_FOR = "Auth failed for "; 117 protected static final String AUTH_SUCCESSFUL_FOR = "Auth successful for "; 118 protected static final Logger AUDITLOG = LoggerFactory.getLogger("SecurityLogger." 119 + Server.class.getName()); 120 protected SecretManager<TokenIdentifier> secretManager; 121 protected final Map<String, String> saslProps; 122 123 protected ServiceAuthorizationManager authManager; 124 125 /** This is set to Call object before Handler invokes an RPC and ybdie 126 * after the call returns. 127 */ 128 protected static final ThreadLocal<RpcCall> CurCall = new ThreadLocal<>(); 129 130 /** Keeps MonitoredRPCHandler per handler thread. */ 131 protected static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC = new ThreadLocal<>(); 132 133 protected final InetSocketAddress bindAddress; 134 135 protected MetricsHBaseServer metrics; 136 137 protected final Configuration conf; 138 139 /** 140 * Maximum size in bytes of the currently queued and running Calls. If a new Call puts us over 141 * this size, then we will reject the call (after parsing it though). It will go back to the 142 * client and client will retry. Set this size with "hbase.ipc.server.max.callqueue.size". The 143 * call queue size gets incremented after we parse a call and before we add it to the queue of 144 * calls for the scheduler to use. It get decremented after we have 'run' the Call. The current 145 * size is kept in {@link #callQueueSizeInBytes}. 146 * @see #callQueueSizeInBytes 147 * @see #DEFAULT_MAX_CALLQUEUE_SIZE 148 */ 149 protected final long maxQueueSizeInBytes; 150 protected static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024; 151 152 /** 153 * This is a running count of the size in bytes of all outstanding calls whether currently 154 * executing or queued waiting to be run. 155 */ 156 protected final LongAdder callQueueSizeInBytes = new LongAdder(); 157 158 protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm 159 protected final boolean tcpKeepAlive; // if T then use keepalives 160 161 /** 162 * This flag is used to indicate to sub threads when they should go down. When we call 163 * {@link #start()}, all threads started will consult this flag on whether they should 164 * keep going. It is set to false when {@link #stop()} is called. 165 */ 166 volatile boolean running = true; 167 168 /** 169 * This flag is set to true after all threads are up and 'running' and the server is then opened 170 * for business by the call to {@link #start()}. 171 */ 172 volatile boolean started = false; 173 174 protected AuthenticationTokenSecretManager authTokenSecretMgr = null; 175 176 protected HBaseRPCErrorHandler errorHandler = null; 177 178 public static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size"; 179 protected static final RequestTooBigException REQUEST_TOO_BIG_EXCEPTION = 180 new RequestTooBigException(); 181 182 protected static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time"; 183 protected static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size"; 184 185 /** 186 * Minimum allowable timeout (in milliseconds) in rpc request's header. This 187 * configuration exists to prevent the rpc service regarding this request as timeout immediately. 188 */ 189 protected static final String MIN_CLIENT_REQUEST_TIMEOUT = "hbase.ipc.min.client.request.timeout"; 190 protected static final int DEFAULT_MIN_CLIENT_REQUEST_TIMEOUT = 20; 191 192 /** Default value for above params */ 193 public static final int DEFAULT_MAX_REQUEST_SIZE = DEFAULT_MAX_CALLQUEUE_SIZE / 4; // 256M 194 protected static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds 195 protected static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024; 196 197 protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000; 198 protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length"; 199 protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>"; 200 201 protected static final Gson GSON = GsonUtil.createGson().create(); 202 203 protected final int maxRequestSize; 204 protected final int warnResponseTime; 205 protected final int warnResponseSize; 206 207 protected final int minClientRequestTimeout; 208 209 protected final Server server; 210 protected final List<BlockingServiceAndInterface> services; 211 212 protected final RpcScheduler scheduler; 213 214 protected UserProvider userProvider; 215 216 protected final ByteBuffAllocator bbAllocator; 217 218 protected volatile boolean allowFallbackToSimpleAuth; 219 220 /** 221 * Used to get details for scan with a scanner_id<br/> 222 * TODO try to figure out a better way and remove reference from regionserver package later. 223 */ 224 private RSRpcServices rsRpcServices; 225 226 227 /** 228 * Use to add online slowlog responses 229 */ 230 private SlowLogRecorder slowLogRecorder; 231 232 @FunctionalInterface 233 protected interface CallCleanup { 234 void run(); 235 } 236 237 /** 238 * Datastructure for passing a {@link BlockingService} and its associated class of 239 * protobuf service interface. For example, a server that fielded what is defined 240 * in the client protobuf service would pass in an implementation of the client blocking service 241 * and then its ClientService.BlockingInterface.class. Used checking connection setup. 242 */ 243 public static class BlockingServiceAndInterface { 244 private final BlockingService service; 245 private final Class<?> serviceInterface; 246 public BlockingServiceAndInterface(final BlockingService service, 247 final Class<?> serviceInterface) { 248 this.service = service; 249 this.serviceInterface = serviceInterface; 250 } 251 public Class<?> getServiceInterface() { 252 return this.serviceInterface; 253 } 254 public BlockingService getBlockingService() { 255 return this.service; 256 } 257 } 258 259 /** 260 * Constructs a server listening on the named port and address. 261 * @param server hosting instance of {@link Server}. We will do authentications if an 262 * instance else pass null for no authentication check. 263 * @param name Used keying this rpc servers' metrics and for naming the Listener thread. 264 * @param services A list of services. 265 * @param bindAddress Where to listen 266 * @param conf 267 * @param scheduler 268 * @param reservoirEnabled Enable ByteBufferPool or not. 269 */ 270 public RpcServer(final Server server, final String name, 271 final List<BlockingServiceAndInterface> services, 272 final InetSocketAddress bindAddress, Configuration conf, 273 RpcScheduler scheduler, boolean reservoirEnabled) throws IOException { 274 this.bbAllocator = ByteBuffAllocator.create(conf, reservoirEnabled); 275 this.server = server; 276 this.services = services; 277 this.bindAddress = bindAddress; 278 this.conf = conf; 279 // See declaration above for documentation on what this size is. 280 this.maxQueueSizeInBytes = 281 this.conf.getLong("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE); 282 283 this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME); 284 this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE); 285 this.minClientRequestTimeout = conf.getInt(MIN_CLIENT_REQUEST_TIMEOUT, 286 DEFAULT_MIN_CLIENT_REQUEST_TIMEOUT); 287 this.maxRequestSize = conf.getInt(MAX_REQUEST_SIZE, DEFAULT_MAX_REQUEST_SIZE); 288 289 this.metrics = new MetricsHBaseServer(name, new MetricsHBaseServerWrapperImpl(this)); 290 this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true); 291 this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true); 292 293 this.cellBlockBuilder = new CellBlockBuilder(conf); 294 295 this.authorize = conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false); 296 this.userProvider = UserProvider.instantiate(conf); 297 this.isSecurityEnabled = userProvider.isHBaseSecurityEnabled(); 298 if (isSecurityEnabled) { 299 saslProps = SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection", 300 QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT))); 301 } else { 302 saslProps = Collections.emptyMap(); 303 } 304 305 this.scheduler = scheduler; 306 } 307 308 @Override 309 public void onConfigurationChange(Configuration newConf) { 310 initReconfigurable(newConf); 311 if (scheduler instanceof ConfigurationObserver) { 312 ((ConfigurationObserver) scheduler).onConfigurationChange(newConf); 313 } 314 if (authorize) { 315 refreshAuthManager(newConf, new HBasePolicyProvider()); 316 } 317 } 318 319 protected void initReconfigurable(Configuration confToLoad) { 320 this.allowFallbackToSimpleAuth = confToLoad.getBoolean(FALLBACK_TO_INSECURE_CLIENT_AUTH, false); 321 if (isSecurityEnabled && allowFallbackToSimpleAuth) { 322 LOG.warn("********* WARNING! *********"); 323 LOG.warn("This server is configured to allow connections from INSECURE clients"); 324 LOG.warn("(" + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = true)."); 325 LOG.warn("While this option is enabled, client identities cannot be secured, and user"); 326 LOG.warn("impersonation is possible!"); 327 LOG.warn("For secure operation, please disable SIMPLE authentication as soon as possible,"); 328 LOG.warn("by setting " + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = false in hbase-site.xml"); 329 LOG.warn("****************************"); 330 } 331 } 332 333 Configuration getConf() { 334 return conf; 335 } 336 337 @Override 338 public boolean isStarted() { 339 return this.started; 340 } 341 342 @Override 343 public synchronized void refreshAuthManager(Configuration conf, PolicyProvider pp) { 344 // Ignore warnings that this should be accessed in a static way instead of via an instance; 345 // it'll break if you go via static route. 346 System.setProperty("hadoop.policy.file", "hbase-policy.xml"); 347 this.authManager.refresh(conf, pp); 348 LOG.info("Refreshed hbase-policy.xml successfully"); 349 ProxyUsers.refreshSuperUserGroupsConfiguration(conf); 350 LOG.info("Refreshed super and proxy users successfully"); 351 } 352 353 protected AuthenticationTokenSecretManager createSecretManager() { 354 if (!isSecurityEnabled) return null; 355 if (server == null) return null; 356 Configuration conf = server.getConfiguration(); 357 long keyUpdateInterval = 358 conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000); 359 long maxAge = 360 conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000); 361 return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(), 362 server.getServerName().toString(), keyUpdateInterval, maxAge); 363 } 364 365 public SecretManager<? extends TokenIdentifier> getSecretManager() { 366 return this.secretManager; 367 } 368 369 @SuppressWarnings("unchecked") 370 public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) { 371 this.secretManager = (SecretManager<TokenIdentifier>) secretManager; 372 } 373 374 /** 375 * This is a server side method, which is invoked over RPC. On success 376 * the return response has protobuf response payload. On failure, the 377 * exception name and the stack trace are returned in the protobuf response. 378 */ 379 @Override 380 public Pair<Message, CellScanner> call(RpcCall call, 381 MonitoredRPCHandler status) throws IOException { 382 try { 383 MethodDescriptor md = call.getMethod(); 384 Message param = call.getParam(); 385 status.setRPC(md.getName(), new Object[]{param}, 386 call.getReceiveTime()); 387 // TODO: Review after we add in encoded data blocks. 388 status.setRPCPacket(param); 389 status.resume("Servicing call"); 390 //get an instance of the method arg type 391 HBaseRpcController controller = new HBaseRpcControllerImpl(call.getCellScanner()); 392 controller.setCallTimeout(call.getTimeout()); 393 Message result = call.getService().callBlockingMethod(md, controller, param); 394 long receiveTime = call.getReceiveTime(); 395 long startTime = call.getStartTime(); 396 long endTime = System.currentTimeMillis(); 397 int processingTime = (int) (endTime - startTime); 398 int qTime = (int) (startTime - receiveTime); 399 int totalTime = (int) (endTime - receiveTime); 400 if (LOG.isTraceEnabled()) { 401 LOG.trace(CurCall.get().toString() + 402 ", response " + TextFormat.shortDebugString(result) + 403 " queueTime: " + qTime + 404 " processingTime: " + processingTime + 405 " totalTime: " + totalTime); 406 } 407 // Use the raw request call size for now. 408 long requestSize = call.getSize(); 409 long responseSize = result.getSerializedSize(); 410 if (call.isClientCellBlockSupported()) { 411 // Include the payload size in HBaseRpcController 412 responseSize += call.getResponseCellSize(); 413 } 414 415 metrics.dequeuedCall(qTime); 416 metrics.processedCall(processingTime); 417 metrics.totalCall(totalTime); 418 metrics.receivedRequest(requestSize); 419 metrics.sentResponse(responseSize); 420 // log any RPC responses that are slower than the configured warn 421 // response time or larger than configured warning size 422 boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1); 423 boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1); 424 if (tooSlow || tooLarge) { 425 final String userName = call.getRequestUserName().orElse(StringUtils.EMPTY); 426 // when tagging, we let TooLarge trump TooSmall to keep output simple 427 // note that large responses will often also be slow. 428 logResponse(param, 429 md.getName(), md.getName() + "(" + param.getClass().getName() + ")", 430 tooLarge, tooSlow, 431 status.getClient(), startTime, processingTime, qTime, 432 responseSize, userName); 433 if (this.slowLogRecorder != null) { 434 // send logs to ring buffer owned by slowLogRecorder 435 final String className = server == null ? StringUtils.EMPTY : 436 server.getClass().getSimpleName(); 437 this.slowLogRecorder.addSlowLogPayload( 438 new RpcLogDetails(call, param, status.getClient(), responseSize, className, tooSlow, 439 tooLarge)); 440 } 441 } 442 return new Pair<>(result, controller.cellScanner()); 443 } catch (Throwable e) { 444 // The above callBlockingMethod will always return a SE. Strip the SE wrapper before 445 // putting it on the wire. Its needed to adhere to the pb Service Interface but we don't 446 // need to pass it over the wire. 447 if (e instanceof ServiceException) { 448 if (e.getCause() == null) { 449 LOG.debug("Caught a ServiceException with null cause", e); 450 } else { 451 e = e.getCause(); 452 } 453 } 454 455 // increment the number of requests that were exceptions. 456 metrics.exception(e); 457 458 if (e instanceof LinkageError) throw new DoNotRetryIOException(e); 459 if (e instanceof IOException) throw (IOException)e; 460 LOG.error("Unexpected throwable object ", e); 461 throw new IOException(e.getMessage(), e); 462 } 463 } 464 465 /** 466 * Logs an RPC response to the LOG file, producing valid JSON objects for 467 * client Operations. 468 * @param param The parameters received in the call. 469 * @param methodName The name of the method invoked 470 * @param call The string representation of the call 471 * @param tooLarge To indicate if the event is tooLarge 472 * @param tooSlow To indicate if the event is tooSlow 473 * @param clientAddress The address of the client who made this call. 474 * @param startTime The time that the call was initiated, in ms. 475 * @param processingTime The duration that the call took to run, in ms. 476 * @param qTime The duration that the call spent on the queue 477 * prior to being initiated, in ms. 478 * @param responseSize The size in bytes of the response buffer. 479 * @param userName UserName of the current RPC Call 480 */ 481 void logResponse(Message param, String methodName, String call, boolean tooLarge, 482 boolean tooSlow, String clientAddress, long startTime, int processingTime, int qTime, 483 long responseSize, String userName) { 484 final String className = server == null ? StringUtils.EMPTY : 485 server.getClass().getSimpleName(); 486 // base information that is reported regardless of type of call 487 Map<String, Object> responseInfo = new HashMap<>(); 488 responseInfo.put("starttimems", startTime); 489 responseInfo.put("processingtimems", processingTime); 490 responseInfo.put("queuetimems", qTime); 491 responseInfo.put("responsesize", responseSize); 492 responseInfo.put("client", clientAddress); 493 responseInfo.put("class", className); 494 responseInfo.put("method", methodName); 495 responseInfo.put("call", call); 496 // The params could be really big, make sure they don't kill us at WARN 497 String stringifiedParam = ProtobufUtil.getShortTextFormat(param); 498 if (stringifiedParam.length() > 150) { 499 // Truncate to 1000 chars if TRACE is on, else to 150 chars 500 stringifiedParam = truncateTraceLog(stringifiedParam); 501 } 502 responseInfo.put("param", stringifiedParam); 503 if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) { 504 ClientProtos.ScanRequest request = ((ClientProtos.ScanRequest) param); 505 String scanDetails; 506 if (request.hasScannerId()) { 507 long scannerId = request.getScannerId(); 508 scanDetails = rsRpcServices.getScanDetailsWithId(scannerId); 509 } else { 510 scanDetails = rsRpcServices.getScanDetailsWithRequest(request); 511 } 512 if (scanDetails != null) { 513 responseInfo.put("scandetails", scanDetails); 514 } 515 } 516 if (param instanceof ClientProtos.MultiRequest) { 517 int numGets = 0; 518 int numMutations = 0; 519 int numServiceCalls = 0; 520 ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest)param; 521 for (ClientProtos.RegionAction regionAction : multi.getRegionActionList()) { 522 for (ClientProtos.Action action: regionAction.getActionList()) { 523 if (action.hasMutation()) { 524 numMutations++; 525 } 526 if (action.hasGet()) { 527 numGets++; 528 } 529 if (action.hasServiceCall()) { 530 numServiceCalls++; 531 } 532 } 533 } 534 responseInfo.put(MULTI_GETS, numGets); 535 responseInfo.put(MULTI_MUTATIONS, numMutations); 536 responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls); 537 } 538 final String tag = (tooLarge && tooSlow) ? "TooLarge & TooSlow" 539 : (tooSlow ? "TooSlow" : "TooLarge"); 540 LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo)); 541 } 542 543 544 /** 545 * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length 546 * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942 547 * @param strParam stringifiedParam to be truncated 548 * @return truncated trace log string 549 */ 550 @VisibleForTesting 551 String truncateTraceLog(String strParam) { 552 if (LOG.isTraceEnabled()) { 553 int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH); 554 int truncatedLength = 555 strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength; 556 String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED; 557 return strParam.subSequence(0, truncatedLength) + truncatedFlag; 558 } 559 return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED; 560 } 561 562 /** 563 * Set the handler for calling out of RPC for error conditions. 564 * @param handler the handler implementation 565 */ 566 @Override 567 public void setErrorHandler(HBaseRPCErrorHandler handler) { 568 this.errorHandler = handler; 569 } 570 571 @Override 572 public HBaseRPCErrorHandler getErrorHandler() { 573 return this.errorHandler; 574 } 575 576 /** 577 * Returns the metrics instance for reporting RPC call statistics 578 */ 579 @Override 580 public MetricsHBaseServer getMetrics() { 581 return metrics; 582 } 583 584 @Override 585 public void addCallSize(final long diff) { 586 this.callQueueSizeInBytes.add(diff); 587 } 588 589 /** 590 * Authorize the incoming client connection. 591 * @param user client user 592 * @param connection incoming connection 593 * @param addr InetAddress of incoming connection 594 * @throws AuthorizationException when the client isn't authorized to talk the protocol 595 */ 596 public synchronized void authorize(UserGroupInformation user, ConnectionHeader connection, 597 InetAddress addr) throws AuthorizationException { 598 if (authorize) { 599 Class<?> c = getServiceInterface(services, connection.getServiceName()); 600 authManager.authorize(user, c, getConf(), addr); 601 } 602 } 603 604 /** 605 * When the read or write buffer size is larger than this limit, i/o will be 606 * done in chunks of this size. Most RPC requests and responses would be 607 * be smaller. 608 */ 609 protected static final int NIO_BUFFER_LIMIT = 64 * 1024; //should not be more than 64KB. 610 611 /** 612 * This is a wrapper around {@link java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)}. 613 * If the amount of data is large, it writes to channel in smaller chunks. 614 * This is to avoid jdk from creating many direct buffers as the size of 615 * ByteBuffer increases. There should not be any performance degredation. 616 * 617 * @param channel writable byte channel to write on 618 * @param buffer buffer to write 619 * @return number of bytes written 620 * @throws java.io.IOException e 621 * @see java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer) 622 */ 623 protected int channelRead(ReadableByteChannel channel, 624 ByteBuffer buffer) throws IOException { 625 626 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ? 627 channel.read(buffer) : channelIO(channel, null, buffer); 628 if (count > 0) { 629 metrics.receivedBytes(count); 630 } 631 return count; 632 } 633 634 /** 635 * Helper for {@link #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)}. 636 * Only one of readCh or writeCh should be non-null. 637 * 638 * @param readCh read channel 639 * @param writeCh write channel 640 * @param buf buffer to read or write into/out of 641 * @return bytes written 642 * @throws java.io.IOException e 643 * @see #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer) 644 */ 645 private static int channelIO(ReadableByteChannel readCh, 646 WritableByteChannel writeCh, 647 ByteBuffer buf) throws IOException { 648 649 int originalLimit = buf.limit(); 650 int initialRemaining = buf.remaining(); 651 int ret = 0; 652 653 while (buf.remaining() > 0) { 654 try { 655 int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT); 656 buf.limit(buf.position() + ioSize); 657 658 ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf); 659 660 if (ret < ioSize) { 661 break; 662 } 663 664 } finally { 665 buf.limit(originalLimit); 666 } 667 } 668 669 int nBytes = initialRemaining - buf.remaining(); 670 return (nBytes > 0) ? nBytes : ret; 671 } 672 673 /** 674 * Needed for features such as delayed calls. We need to be able to store the current call 675 * so that we can complete it later or ask questions of what is supported by the current ongoing 676 * call. 677 * @return An RpcCallContext backed by the currently ongoing call (gotten from a thread local) 678 */ 679 public static Optional<RpcCall> getCurrentCall() { 680 return Optional.ofNullable(CurCall.get()); 681 } 682 683 public static boolean isInRpcCallContext() { 684 return CurCall.get() != null; 685 } 686 687 /** 688 * Used by {@link org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore}. For 689 * master's rpc call, it may generate new procedure and mutate the region which store procedure. 690 * There are some check about rpc when mutate region, such as rpc timeout check. So unset the rpc 691 * call to avoid the rpc check. 692 * @return the currently ongoing rpc call 693 */ 694 public static Optional<RpcCall> unsetCurrentCall() { 695 Optional<RpcCall> rpcCall = getCurrentCall(); 696 CurCall.set(null); 697 return rpcCall; 698 } 699 700 /** 701 * Used by {@link org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore}. Set the 702 * rpc call back after mutate region. 703 */ 704 public static void setCurrentCall(RpcCall rpcCall) { 705 CurCall.set(rpcCall); 706 } 707 708 /** 709 * Returns the user credentials associated with the current RPC request or not present if no 710 * credentials were provided. 711 * @return A User 712 */ 713 public static Optional<User> getRequestUser() { 714 Optional<RpcCall> ctx = getCurrentCall(); 715 return ctx.isPresent() ? ctx.get().getRequestUser() : Optional.empty(); 716 } 717 718 /** 719 * The number of open RPC conections 720 * @return the number of open rpc connections 721 */ 722 abstract public int getNumOpenConnections(); 723 724 /** 725 * Returns the username for any user associated with the current RPC 726 * request or not present if no user is set. 727 */ 728 public static Optional<String> getRequestUserName() { 729 return getRequestUser().map(User::getShortName); 730 } 731 732 /** 733 * @return Address of remote client if a request is ongoing, else null 734 */ 735 public static Optional<InetAddress> getRemoteAddress() { 736 return getCurrentCall().map(RpcCall::getRemoteAddress); 737 } 738 739 /** 740 * @param serviceName Some arbitrary string that represents a 'service'. 741 * @param services Available service instances 742 * @return Matching BlockingServiceAndInterface pair 743 */ 744 protected static BlockingServiceAndInterface getServiceAndInterface( 745 final List<BlockingServiceAndInterface> services, final String serviceName) { 746 for (BlockingServiceAndInterface bs : services) { 747 if (bs.getBlockingService().getDescriptorForType().getName().equals(serviceName)) { 748 return bs; 749 } 750 } 751 return null; 752 } 753 754 /** 755 * @param serviceName Some arbitrary string that represents a 'service'. 756 * @param services Available services and their service interfaces. 757 * @return Service interface class for <code>serviceName</code> 758 */ 759 protected static Class<?> getServiceInterface( 760 final List<BlockingServiceAndInterface> services, 761 final String serviceName) { 762 BlockingServiceAndInterface bsasi = 763 getServiceAndInterface(services, serviceName); 764 return bsasi == null? null: bsasi.getServiceInterface(); 765 } 766 767 /** 768 * @param serviceName Some arbitrary string that represents a 'service'. 769 * @param services Available services and their service interfaces. 770 * @return BlockingService that goes with the passed <code>serviceName</code> 771 */ 772 protected static BlockingService getService( 773 final List<BlockingServiceAndInterface> services, 774 final String serviceName) { 775 BlockingServiceAndInterface bsasi = 776 getServiceAndInterface(services, serviceName); 777 return bsasi == null? null: bsasi.getBlockingService(); 778 } 779 780 protected static MonitoredRPCHandler getStatus() { 781 // It is ugly the way we park status up in RpcServer. Let it be for now. TODO. 782 MonitoredRPCHandler status = RpcServer.MONITORED_RPC.get(); 783 if (status != null) { 784 return status; 785 } 786 status = TaskMonitor.get().createRPCStatus(Thread.currentThread().getName()); 787 status.pause("Waiting for a call"); 788 RpcServer.MONITORED_RPC.set(status); 789 return status; 790 } 791 792 /** Returns the remote side ip address when invoked inside an RPC 793 * Returns null incase of an error. 794 * @return InetAddress 795 */ 796 public static InetAddress getRemoteIp() { 797 RpcCall call = CurCall.get(); 798 if (call != null) { 799 return call.getRemoteAddress(); 800 } 801 return null; 802 } 803 804 @Override 805 public RpcScheduler getScheduler() { 806 return scheduler; 807 } 808 809 @Override 810 public ByteBuffAllocator getByteBuffAllocator() { 811 return this.bbAllocator; 812 } 813 814 @Override 815 public void setRsRpcServices(RSRpcServices rsRpcServices) { 816 this.rsRpcServices = rsRpcServices; 817 } 818 819 @Override 820 public void setSlowLogRecorder(SlowLogRecorder slowLogRecorder) { 821 this.slowLogRecorder = slowLogRecorder; 822 } 823 824 @Override 825 public SlowLogRecorder getSlowLogRecorder() { 826 return slowLogRecorder; 827 } 828}