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}