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}