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 static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader; 021import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException; 022import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited; 023import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException; 024import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled; 025import static org.apache.hadoop.hbase.ipc.IPCUtil.write; 026 027import java.io.BufferedInputStream; 028import java.io.BufferedOutputStream; 029import java.io.DataInputStream; 030import java.io.DataOutputStream; 031import java.io.IOException; 032import java.io.InputStream; 033import java.io.InterruptedIOException; 034import java.io.OutputStream; 035import java.net.Socket; 036import java.net.SocketTimeoutException; 037import java.net.UnknownHostException; 038import java.security.PrivilegedExceptionAction; 039import java.util.ArrayDeque; 040import java.util.Locale; 041import java.util.Queue; 042import java.util.concurrent.ConcurrentHashMap; 043import java.util.concurrent.ConcurrentMap; 044import java.util.concurrent.ThreadLocalRandom; 045import javax.security.sasl.SaslException; 046 047import org.apache.hadoop.conf.Configuration; 048import org.apache.hadoop.hbase.CellScanner; 049import org.apache.hadoop.hbase.DoNotRetryIOException; 050import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; 051import org.apache.hadoop.hbase.io.ByteArrayOutputStream; 052import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback; 053import org.apache.hadoop.hbase.log.HBaseMarkers; 054import org.apache.hadoop.hbase.security.HBaseSaslRpcClient; 055import org.apache.hadoop.hbase.security.SaslUtil; 056import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection; 057import org.apache.hadoop.hbase.trace.TraceUtil; 058import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 059import org.apache.hadoop.hbase.util.ExceptionUtil; 060import org.apache.hadoop.io.IOUtils; 061import org.apache.hadoop.ipc.RemoteException; 062import org.apache.hadoop.net.NetUtils; 063import org.apache.hadoop.security.UserGroupInformation; 064import org.apache.hadoop.util.StringUtils; 065import org.apache.htrace.core.TraceScope; 066import org.apache.yetus.audience.InterfaceAudience; 067import org.slf4j.Logger; 068import org.slf4j.LoggerFactory; 069import org.apache.hbase.thirdparty.com.google.protobuf.Message; 070import org.apache.hbase.thirdparty.com.google.protobuf.Message.Builder; 071import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; 072import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; 073import org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator; 074import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 075import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; 076import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; 077import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; 078import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; 079import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; 080import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader; 081 082/** 083 * Thread that reads responses and notifies callers. Each connection owns a socket connected to a 084 * remote address. Calls are multiplexed through this socket: responses may be delivered out of 085 * order. 086 */ 087@InterfaceAudience.Private 088class BlockingRpcConnection extends RpcConnection implements Runnable { 089 090 private static final Logger LOG = LoggerFactory.getLogger(BlockingRpcConnection.class); 091 092 private final BlockingRpcClient rpcClient; 093 094 private final String threadName; 095 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC", 096 justification = "We are always under lock actually") 097 private Thread thread; 098 099 // connected socket. protected for writing UT. 100 protected Socket socket = null; 101 private DataInputStream in; 102 private DataOutputStream out; 103 104 private HBaseSaslRpcClient saslRpcClient; 105 106 // currently active calls 107 private final ConcurrentMap<Integer, Call> calls = new ConcurrentHashMap<>(); 108 109 private final CallSender callSender; 110 111 private boolean closed = false; 112 113 private byte[] connectionHeaderPreamble; 114 115 private byte[] connectionHeaderWithLength; 116 117 private boolean waitingConnectionHeaderResponse = false; 118 119 /** 120 * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a 121 * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to 122 * use a different thread for writing. This way, on interruptions, we either cancel the writes or 123 * ignore the answer if the write is already done, but we don't stop the write in the middle. This 124 * adds a thread per region server in the client, so it's kept as an option. 125 * <p> 126 * The implementation is simple: the client threads adds their call to the queue, and then wait 127 * for an answer. The CallSender blocks on the queue, and writes the calls one after the other. On 128 * interruption, the client cancels its call. The CallSender checks that the call has not been 129 * canceled before writing it. 130 * </p> 131 * When the connection closes, all the calls not yet sent are dismissed. The client thread is 132 * notified with an appropriate exception, as if the call was already sent but the answer not yet 133 * received. 134 * </p> 135 */ 136 private class CallSender extends Thread { 137 138 private final Queue<Call> callsToWrite; 139 140 private final int maxQueueSize; 141 142 public CallSender(String name, Configuration conf) { 143 int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000); 144 callsToWrite = new ArrayDeque<>(queueSize); 145 this.maxQueueSize = queueSize; 146 setDaemon(true); 147 setName(name + " - writer"); 148 } 149 150 public void sendCall(final Call call) throws IOException { 151 if (callsToWrite.size() >= maxQueueSize) { 152 throw new IOException("Can't add " + call.toShortString() 153 + " to the write queue. callsToWrite.size()=" + callsToWrite.size()); 154 } 155 callsToWrite.offer(call); 156 BlockingRpcConnection.this.notifyAll(); 157 } 158 159 public void remove(Call call) { 160 callsToWrite.remove(call); 161 // By removing the call from the expected call list, we make the list smaller, but 162 // it means as well that we don't know how many calls we cancelled. 163 calls.remove(call.id); 164 call.setException(new CallCancelledException(call.toShortString() + ", waitTime=" 165 + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout=" 166 + call.timeout)); 167 } 168 169 /** 170 * Reads the call from the queue, write them on the socket. 171 */ 172 @Override 173 public void run() { 174 synchronized (BlockingRpcConnection.this) { 175 while (!closed) { 176 if (callsToWrite.isEmpty()) { 177 // We should use another monitor object here for better performance since the read 178 // thread also uses ConnectionImpl.this. But this makes the locking schema more 179 // complicated, can do it later as an optimization. 180 try { 181 BlockingRpcConnection.this.wait(); 182 } catch (InterruptedException e) { 183 } 184 // check if we need to quit, so continue the main loop instead of fallback. 185 continue; 186 } 187 Call call = callsToWrite.poll(); 188 if (call.isDone()) { 189 continue; 190 } 191 try { 192 tracedWriteRequest(call); 193 } catch (IOException e) { 194 // exception here means the call has not been added to the pendingCalls yet, so we need 195 // to fail it by our own. 196 LOG.debug("call write error for {}", call.toShortString()); 197 call.setException(e); 198 closeConn(e); 199 } 200 } 201 } 202 } 203 204 /** 205 * Cleans the call not yet sent when we finish. 206 */ 207 public void cleanup(IOException e) { 208 IOException ie = new ConnectionClosingException( 209 "Connection to " + remoteId.address + " is closing."); 210 for (Call call : callsToWrite) { 211 call.setException(ie); 212 } 213 callsToWrite.clear(); 214 } 215 } 216 217 BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException { 218 super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, 219 rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor); 220 this.rpcClient = rpcClient; 221 if (remoteId.getAddress().isUnresolved()) { 222 throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName()); 223 } 224 225 this.connectionHeaderPreamble = getConnectionHeaderPreamble(); 226 ConnectionHeader header = getConnectionHeader(); 227 ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize()); 228 DataOutputStream dos = new DataOutputStream(baos); 229 dos.writeInt(header.getSerializedSize()); 230 header.writeTo(dos); 231 assert baos.size() == 4 + header.getSerializedSize(); 232 this.connectionHeaderWithLength = baos.getBuffer(); 233 234 UserGroupInformation ticket = remoteId.ticket.getUGI(); 235 this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() + ") connection to " 236 + remoteId.getAddress().toString() 237 + ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName())); 238 239 if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) { 240 callSender = new CallSender(threadName, this.rpcClient.conf); 241 callSender.start(); 242 } else { 243 callSender = null; 244 } 245 } 246 247 // protected for write UT. 248 protected void setupConnection() throws IOException { 249 short ioFailures = 0; 250 short timeoutFailures = 0; 251 while (true) { 252 try { 253 this.socket = this.rpcClient.socketFactory.createSocket(); 254 this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay()); 255 this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive); 256 if (this.rpcClient.localAddr != null) { 257 this.socket.bind(this.rpcClient.localAddr); 258 } 259 NetUtils.connect(this.socket, remoteId.getAddress(), this.rpcClient.connectTO); 260 this.socket.setSoTimeout(this.rpcClient.readTO); 261 return; 262 } catch (SocketTimeoutException toe) { 263 /* 264 * The max number of retries is 45, which amounts to 20s*45 = 15 minutes retries. 265 */ 266 if (LOG.isDebugEnabled()) { 267 LOG.debug("Received exception in connection setup.\n" + 268 StringUtils.stringifyException(toe)); 269 } 270 handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe); 271 } catch (IOException ie) { 272 if (LOG.isDebugEnabled()) { 273 LOG.debug("Received exception in connection setup.\n" + 274 StringUtils.stringifyException(ie)); 275 } 276 handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie); 277 } 278 } 279 } 280 281 /** 282 * Handle connection failures If the current number of retries is equal to the max number of 283 * retries, stop retrying and throw the exception; Otherwise backoff N seconds and try connecting 284 * again. This Method is only called from inside setupIOstreams(), which is synchronized. Hence 285 * the sleep is synchronized; the locks will be retained. 286 * @param curRetries current number of retries 287 * @param maxRetries max number of retries allowed 288 * @param ioe failure reason 289 * @throws IOException if max number of retries is reached 290 */ 291 private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe) 292 throws IOException { 293 closeSocket(); 294 295 // throw the exception if the maximum number of retries is reached 296 if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) { 297 throw ioe; 298 } 299 300 // otherwise back off and retry 301 try { 302 Thread.sleep(this.rpcClient.failureSleep); 303 } catch (InterruptedException ie) { 304 ExceptionUtil.rethrowIfInterrupt(ie); 305 } 306 307 if (LOG.isInfoEnabled()) { 308 LOG.info("Retrying connect to server: " + remoteId.getAddress() + 309 " after sleeping " + this.rpcClient.failureSleep + "ms. Already tried " + curRetries + 310 " time(s)."); 311 } 312 } 313 314 /* 315 * wait till someone signals us to start reading RPC response or it is idle too long, it is marked 316 * as to be closed, or the client is marked as not running. 317 * @return true if it is time to read a response; false otherwise. 318 */ 319 private synchronized boolean waitForWork() { 320 // beware of the concurrent access to the calls list: we can add calls, but as well 321 // remove them. 322 long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose; 323 for (;;) { 324 if (thread == null) { 325 return false; 326 } 327 if (!calls.isEmpty()) { 328 return true; 329 } 330 if (EnvironmentEdgeManager.currentTime() >= waitUntil) { 331 closeConn( 332 new IOException("idle connection closed with " + calls.size() + " pending request(s)")); 333 return false; 334 } 335 try { 336 wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000)); 337 } catch (InterruptedException e) { 338 } 339 } 340 } 341 342 @Override 343 public void run() { 344 if (LOG.isTraceEnabled()) { 345 LOG.trace(threadName + ": starting, connections " + this.rpcClient.connections.size()); 346 } 347 while (waitForWork()) { 348 readResponse(); 349 } 350 if (LOG.isTraceEnabled()) { 351 LOG.trace(threadName + ": stopped, connections " + this.rpcClient.connections.size()); 352 } 353 } 354 355 private void disposeSasl() { 356 if (saslRpcClient != null) { 357 saslRpcClient.dispose(); 358 saslRpcClient = null; 359 } 360 } 361 362 private boolean setupSaslConnection(final InputStream in2, final OutputStream out2) 363 throws IOException { 364 saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, 365 this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection", 366 QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)), 367 this.rpcClient.conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT)); 368 return saslRpcClient.saslConnect(in2, out2); 369 } 370 371 /** 372 * If multiple clients with the same principal try to connect to the same server at the same time, 373 * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to 374 * work around this, what is done is that the client backs off randomly and tries to initiate the 375 * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is 376 * attempted. 377 * <p> 378 * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the 379 * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such 380 * cases, it is prudent to throw a runtime exception when we receive a SaslException from the 381 * underlying authentication implementation, so there is no retry from other high level (for eg, 382 * HCM or HBaseAdmin). 383 * </p> 384 */ 385 private void handleSaslConnectionFailure(final int currRetries, final int maxRetries, 386 final Exception ex, final UserGroupInformation user) 387 throws IOException, InterruptedException { 388 closeSocket(); 389 user.doAs(new PrivilegedExceptionAction<Object>() { 390 @Override 391 public Object run() throws IOException, InterruptedException { 392 if (shouldAuthenticateOverKrb()) { 393 if (currRetries < maxRetries) { 394 if (LOG.isDebugEnabled()) { 395 LOG.debug("Exception encountered while connecting to " + 396 "the server : " + StringUtils.stringifyException(ex)); 397 } 398 // try re-login 399 relogin(); 400 disposeSasl(); 401 // have granularity of milliseconds 402 // we are sleeping with the Connection lock held but since this 403 // connection instance is being used for connecting to the server 404 // in question, it is okay 405 Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1); 406 return null; 407 } else { 408 String msg = "Couldn't setup connection for " 409 + UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal; 410 LOG.warn(msg, ex); 411 throw (IOException) new IOException(msg).initCause(ex); 412 } 413 } else { 414 LOG.warn("Exception encountered while connecting to " + "the server : " + ex); 415 } 416 if (ex instanceof RemoteException) { 417 throw (RemoteException) ex; 418 } 419 if (ex instanceof SaslException) { 420 String msg = "SASL authentication failed." 421 + " The most likely cause is missing or invalid credentials." + " Consider 'kinit'."; 422 LOG.error(HBaseMarkers.FATAL, msg, ex); 423 throw new RuntimeException(msg, ex); 424 } 425 throw new IOException(ex); 426 } 427 }); 428 } 429 430 private void setupIOstreams() throws IOException { 431 if (socket != null) { 432 // The connection is already available. Perfect. 433 return; 434 } 435 436 if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) { 437 if (LOG.isDebugEnabled()) { 438 LOG.debug("Not trying to connect to " + remoteId.address 439 + " this server is in the failed servers list"); 440 } 441 throw new FailedServerException( 442 "This server is in the failed servers list: " + remoteId.address); 443 } 444 445 try { 446 if (LOG.isDebugEnabled()) { 447 LOG.debug("Connecting to " + remoteId.address); 448 } 449 450 short numRetries = 0; 451 final short MAX_RETRIES = 5; 452 while (true) { 453 setupConnection(); 454 InputStream inStream = NetUtils.getInputStream(socket); 455 // This creates a socket with a write timeout. This timeout cannot be changed. 456 OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO); 457 // Write out the preamble -- MAGIC, version, and auth to use. 458 writeConnectionHeaderPreamble(outStream); 459 if (useSasl) { 460 final InputStream in2 = inStream; 461 final OutputStream out2 = outStream; 462 UserGroupInformation ticket = getUGI(); 463 boolean continueSasl; 464 if (ticket == null) { 465 throw new FatalConnectionException("ticket/user is null"); 466 } 467 try { 468 continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() { 469 @Override 470 public Boolean run() throws IOException { 471 return setupSaslConnection(in2, out2); 472 } 473 }); 474 } catch (Exception ex) { 475 ExceptionUtil.rethrowIfInterrupt(ex); 476 handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, ticket); 477 continue; 478 } 479 if (continueSasl) { 480 // Sasl connect is successful. Let's set up Sasl i/o streams. 481 inStream = saslRpcClient.getInputStream(); 482 outStream = saslRpcClient.getOutputStream(); 483 } else { 484 // fall back to simple auth because server told us so. 485 // do not change authMethod and useSasl here, we should start from secure when 486 // reconnecting because regionserver may change its sasl config after restart. 487 } 488 } 489 this.in = new DataInputStream(new BufferedInputStream(inStream)); 490 this.out = new DataOutputStream(new BufferedOutputStream(outStream)); 491 // Now write out the connection header 492 writeConnectionHeader(); 493 // process the response from server for connection header if necessary 494 processResponseForConnectionHeader(); 495 496 break; 497 } 498 } catch (Throwable t) { 499 closeSocket(); 500 IOException e = ExceptionUtil.asInterrupt(t); 501 if (e == null) { 502 this.rpcClient.failedServers.addToFailedServers(remoteId.address, t); 503 if (t instanceof LinkageError) { 504 // probably the hbase hadoop version does not match the running hadoop version 505 e = new DoNotRetryIOException(t); 506 } else if (t instanceof IOException) { 507 e = (IOException) t; 508 } else { 509 e = new IOException("Could not set up IO Streams to " + remoteId.address, t); 510 } 511 } 512 throw e; 513 } 514 515 // start the receiver thread after the socket connection has been set up 516 thread = new Thread(this, threadName); 517 thread.setDaemon(true); 518 thread.start(); 519 } 520 521 /** 522 * Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>} 523 */ 524 private void writeConnectionHeaderPreamble(OutputStream out) throws IOException { 525 out.write(connectionHeaderPreamble); 526 out.flush(); 527 } 528 529 /** 530 * Write the connection header. 531 */ 532 private void writeConnectionHeader() throws IOException { 533 boolean isCryptoAesEnable = false; 534 // check if Crypto AES is enabled 535 if (saslRpcClient != null) { 536 boolean saslEncryptionEnabled = SaslUtil.QualityOfProtection.PRIVACY. 537 getSaslQop().equalsIgnoreCase(saslRpcClient.getSaslQOP()); 538 isCryptoAesEnable = saslEncryptionEnabled && conf.getBoolean( 539 CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT); 540 } 541 542 // if Crypto AES is enabled, set transformation and negotiate with server 543 if (isCryptoAesEnable) { 544 waitingConnectionHeaderResponse = true; 545 } 546 this.out.write(connectionHeaderWithLength); 547 this.out.flush(); 548 } 549 550 private void processResponseForConnectionHeader() throws IOException { 551 // if no response excepted, return 552 if (!waitingConnectionHeaderResponse) return; 553 try { 554 // read the ConnectionHeaderResponse from server 555 int len = this.in.readInt(); 556 byte[] buff = new byte[len]; 557 int readSize = this.in.read(buff); 558 if (LOG.isDebugEnabled()) { 559 LOG.debug("Length of response for connection header:" + readSize); 560 } 561 562 RPCProtos.ConnectionHeaderResponse connectionHeaderResponse = 563 RPCProtos.ConnectionHeaderResponse.parseFrom(buff); 564 565 // Get the CryptoCipherMeta, update the HBaseSaslRpcClient for Crypto Cipher 566 if (connectionHeaderResponse.hasCryptoCipherMeta()) { 567 negotiateCryptoAes(connectionHeaderResponse.getCryptoCipherMeta()); 568 } 569 waitingConnectionHeaderResponse = false; 570 } catch (SocketTimeoutException ste) { 571 LOG.error(HBaseMarkers.FATAL, "Can't get the connection header response for rpc timeout, " 572 + "please check if server has the correct configuration to support the additional " 573 + "function.", ste); 574 // timeout when waiting the connection header response, ignore the additional function 575 throw new IOException("Timeout while waiting connection header response", ste); 576 } 577 } 578 579 private void negotiateCryptoAes(RPCProtos.CryptoCipherMeta cryptoCipherMeta) 580 throws IOException { 581 // initialize the Crypto AES with CryptoCipherMeta 582 saslRpcClient.initCryptoCipher(cryptoCipherMeta, this.rpcClient.conf); 583 // reset the inputStream/outputStream for Crypto AES encryption 584 this.in = new DataInputStream(new BufferedInputStream(saslRpcClient.getInputStream())); 585 this.out = new DataOutputStream(new BufferedOutputStream(saslRpcClient.getOutputStream())); 586 } 587 588 private void tracedWriteRequest(Call call) throws IOException { 589 try (TraceScope ignored = TraceUtil.createTrace("RpcClientImpl.tracedWriteRequest", 590 call.span)) { 591 writeRequest(call); 592 } 593 } 594 595 /** 596 * Initiates a call by sending the parameter to the remote server. Note: this is not called from 597 * the Connection thread, but by other threads. 598 * @see #readResponse() 599 */ 600 private void writeRequest(Call call) throws IOException { 601 ByteBuf cellBlock = null; 602 try { 603 cellBlock = this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec, this.compressor, 604 call.cells, PooledByteBufAllocator.DEFAULT); 605 CellBlockMeta cellBlockMeta; 606 if (cellBlock != null) { 607 cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.readableBytes()).build(); 608 } else { 609 cellBlockMeta = null; 610 } 611 RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta); 612 613 setupIOstreams(); 614 615 // Now we're going to write the call. We take the lock, then check that the connection 616 // is still valid, and, if so we do the write to the socket. If the write fails, we don't 617 // know where we stand, we have to close the connection. 618 if (Thread.interrupted()) { 619 throw new InterruptedIOException(); 620 } 621 622 calls.put(call.id, call); // We put first as we don't want the connection to become idle. 623 // from here, we do not throw any exception to upper layer as the call has been tracked in 624 // the pending calls map. 625 try { 626 call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock)); 627 } catch (Throwable t) { 628 if(LOG.isTraceEnabled()) { 629 LOG.trace("Error while writing {}", call.toShortString()); 630 } 631 IOException e = IPCUtil.toIOE(t); 632 closeConn(e); 633 return; 634 } 635 } finally { 636 if (cellBlock != null) { 637 cellBlock.release(); 638 } 639 } 640 notifyAll(); 641 } 642 643 /* 644 * Receive a response. Because only one receiver, so no synchronization on in. 645 */ 646 private void readResponse() { 647 Call call = null; 648 boolean expectedCall = false; 649 try { 650 // See HBaseServer.Call.setResponse for where we write out the response. 651 // Total size of the response. Unused. But have to read it in anyways. 652 int totalSize = in.readInt(); 653 654 // Read the header 655 ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); 656 int id = responseHeader.getCallId(); 657 call = calls.remove(id); // call.done have to be set before leaving this method 658 expectedCall = (call != null && !call.isDone()); 659 if (!expectedCall) { 660 // So we got a response for which we have no corresponding 'call' here on the client-side. 661 // We probably timed out waiting, cleaned up all references, and now the server decides 662 // to return a response. There is nothing we can do w/ the response at this stage. Clean 663 // out the wire of the response so its out of the way and we can get other responses on 664 // this connection. 665 int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader); 666 int whatIsLeftToRead = totalSize - readSoFar; 667 IOUtils.skipFully(in, whatIsLeftToRead); 668 if (call != null) { 669 call.callStats.setResponseSizeBytes(totalSize); 670 call.callStats 671 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); 672 } 673 return; 674 } 675 if (responseHeader.hasException()) { 676 ExceptionResponse exceptionResponse = responseHeader.getException(); 677 RemoteException re = createRemoteException(exceptionResponse); 678 call.setException(re); 679 call.callStats.setResponseSizeBytes(totalSize); 680 call.callStats 681 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); 682 if (isFatalConnectionException(exceptionResponse)) { 683 synchronized (this) { 684 closeConn(re); 685 } 686 } 687 } else { 688 Message value = null; 689 if (call.responseDefaultType != null) { 690 Builder builder = call.responseDefaultType.newBuilderForType(); 691 ProtobufUtil.mergeDelimitedFrom(builder, in); 692 value = builder.build(); 693 } 694 CellScanner cellBlockScanner = null; 695 if (responseHeader.hasCellBlockMeta()) { 696 int size = responseHeader.getCellBlockMeta().getLength(); 697 byte[] cellBlock = new byte[size]; 698 IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); 699 cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec, 700 this.compressor, cellBlock); 701 } 702 call.setResponse(value, cellBlockScanner); 703 call.callStats.setResponseSizeBytes(totalSize); 704 call.callStats 705 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime()); 706 } 707 } catch (IOException e) { 708 if (expectedCall) { 709 call.setException(e); 710 } 711 if (e instanceof SocketTimeoutException) { 712 // Clean up open calls but don't treat this as a fatal condition, 713 // since we expect certain responses to not make it by the specified 714 // {@link ConnectionId#rpcTimeout}. 715 if (LOG.isTraceEnabled()) { 716 LOG.trace("ignored", e); 717 } 718 } else { 719 synchronized (this) { 720 closeConn(e); 721 } 722 } 723 } 724 } 725 726 @Override 727 protected synchronized void callTimeout(Call call) { 728 // call sender 729 calls.remove(call.id); 730 } 731 732 // just close socket input and output. 733 private void closeSocket() { 734 IOUtils.closeStream(out); 735 IOUtils.closeStream(in); 736 IOUtils.closeSocket(socket); 737 out = null; 738 in = null; 739 socket = null; 740 } 741 742 // close socket, reader, and clean up all pending calls. 743 private void closeConn(IOException e) { 744 if (thread == null) { 745 return; 746 } 747 thread.interrupt(); 748 thread = null; 749 closeSocket(); 750 if (callSender != null) { 751 callSender.cleanup(e); 752 } 753 for (Call call : calls.values()) { 754 call.setException(e); 755 } 756 calls.clear(); 757 } 758 759 // release all resources, the connection will not be used any more. 760 @Override 761 public synchronized void shutdown() { 762 closed = true; 763 if (callSender != null) { 764 callSender.interrupt(); 765 } 766 closeConn(new IOException("connection to " + remoteId.address + " closed")); 767 } 768 769 @Override 770 public void cleanupConnection() { 771 // do nothing 772 } 773 774 @Override 775 public synchronized void sendRequest(final Call call, HBaseRpcController pcrc) 776 throws IOException { 777 pcrc.notifyOnCancel(new RpcCallback<Object>() { 778 779 @Override 780 public void run(Object parameter) { 781 setCancelled(call); 782 synchronized (BlockingRpcConnection.this) { 783 if (callSender != null) { 784 callSender.remove(call); 785 } else { 786 calls.remove(call.id); 787 } 788 } 789 } 790 }, new CancellationCallback() { 791 792 @Override 793 public void run(boolean cancelled) throws IOException { 794 if (cancelled) { 795 setCancelled(call); 796 return; 797 } 798 scheduleTimeoutTask(call); 799 if (callSender != null) { 800 callSender.sendCall(call); 801 } else { 802 tracedWriteRequest(call); 803 } 804 } 805 }); 806 } 807 808 @Override 809 public synchronized boolean isActive() { 810 return thread != null; 811 } 812}