View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.ipc;
19  
20  import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader;
21  import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException;
22  import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited;
23  import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
24  import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
25  import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
26  
27  import com.google.protobuf.Message;
28  import com.google.protobuf.Message.Builder;
29  import com.google.protobuf.RpcCallback;
30  
31  import java.io.BufferedInputStream;
32  import java.io.BufferedOutputStream;
33  import java.io.DataInputStream;
34  import java.io.DataOutputStream;
35  import java.io.IOException;
36  import java.io.InputStream;
37  import java.io.InterruptedIOException;
38  import java.io.OutputStream;
39  import java.net.InetSocketAddress;
40  import java.net.Socket;
41  import java.net.SocketTimeoutException;
42  import java.net.UnknownHostException;
43  import java.nio.ByteBuffer;
44  import java.security.PrivilegedExceptionAction;
45  import java.util.ArrayDeque;
46  import java.util.Locale;
47  import java.util.Queue;
48  import java.util.concurrent.ConcurrentHashMap;
49  import java.util.concurrent.ConcurrentMap;
50  import java.util.concurrent.ThreadLocalRandom;
51  
52  import javax.security.sasl.SaslException;
53  
54  import org.apache.commons.logging.Log;
55  import org.apache.commons.logging.LogFactory;
56  import org.apache.hadoop.conf.Configuration;
57  import org.apache.hadoop.hbase.CellScanner;
58  import org.apache.hadoop.hbase.DoNotRetryIOException;
59  import org.apache.hadoop.hbase.classification.InterfaceAudience;
60  import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
61  import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
62  import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
63  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
64  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
65  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
66  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
67  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
68  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
69  import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
70  import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
71  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
72  import org.apache.hadoop.hbase.util.ExceptionUtil;
73  import org.apache.hadoop.io.IOUtils;
74  import org.apache.hadoop.ipc.RemoteException;
75  import org.apache.hadoop.net.NetUtils;
76  import org.apache.hadoop.security.UserGroupInformation;
77  import org.apache.htrace.Trace;
78  import org.apache.htrace.TraceScope;
79  
80  /**
81   * Thread that reads responses and notifies callers. Each connection owns a socket connected to a
82   * remote address. Calls are multiplexed through this socket: responses may be delivered out of
83   * order.
84   */
85  @InterfaceAudience.Private
86  class BlockingRpcConnection extends RpcConnection implements Runnable {
87  
88    private static final Log LOG = LogFactory.getLog(BlockingRpcConnection.class);
89  
90    private final BlockingRpcClient rpcClient;
91  
92    private final String threadName;
93    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
94        justification = "We are always under lock actually")
95    private Thread thread;
96  
97    // connected socket. protected for writing UT.
98    protected Socket socket = null;
99    private DataInputStream in;
100   private DataOutputStream out;
101 
102   private HBaseSaslRpcClient saslRpcClient;
103 
104   // currently active calls
105   private final ConcurrentMap<Integer, Call> calls = new ConcurrentHashMap<>();
106 
107   private final CallSender callSender;
108 
109   private boolean closed = false;
110 
111   private byte[] connectionHeaderPreamble;
112 
113   private byte[] connectionHeaderWithLength;
114 
115   /**
116    * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a
117    * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to
118    * use a different thread for writing. This way, on interruptions, we either cancel the writes or
119    * ignore the answer if the write is already done, but we don't stop the write in the middle. This
120    * adds a thread per region server in the client, so it's kept as an option.
121    * <p>
122    * The implementation is simple: the client threads adds their call to the queue, and then wait
123    * for an answer. The CallSender blocks on the queue, and writes the calls one after the other. On
124    * interruption, the client cancels its call. The CallSender checks that the call has not been
125    * canceled before writing it.
126    * </p>
127    * When the connection closes, all the calls not yet sent are dismissed. The client thread is
128    * notified with an appropriate exception, as if the call was already sent but the answer not yet
129    * received.
130    * </p>
131    */
132   private class CallSender extends Thread {
133 
134     private final Queue<Call> callsToWrite;
135 
136     private final int maxQueueSize;
137 
138     public CallSender(String name, Configuration conf) {
139       int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
140       callsToWrite = new ArrayDeque<>(queueSize);
141       this.maxQueueSize = queueSize;
142       setDaemon(true);
143       setName(name + " - writer");
144     }
145 
146     public void sendCall(final Call call) throws IOException {
147       if (callsToWrite.size() >= maxQueueSize) {
148         throw new IOException("Can't add the call " + call.id +
149             " to the write queue. callsToWrite.size()=" + callsToWrite.size());
150       }
151       callsToWrite.offer(call);
152       BlockingRpcConnection.this.notifyAll();
153     }
154 
155     public void remove(Call call) {
156       callsToWrite.remove(call);
157       // By removing the call from the expected call list, we make the list smaller, but
158       // it means as well that we don't know how many calls we cancelled.
159       calls.remove(call.id);
160       call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime=" +
161           (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout=" +
162           call.timeout));
163     }
164 
165     /**
166      * Reads the call from the queue, write them on the socket.
167      */
168     @Override
169     public void run() {
170       synchronized (BlockingRpcConnection.this) {
171         while (!closed) {
172           if (callsToWrite.isEmpty()) {
173             // We should use another monitor object here for better performance since the read
174             // thread also uses ConnectionImpl.this. But this makes the locking schema more
175             // complicated, can do it later as an optimization.
176             try {
177               BlockingRpcConnection.this.wait();
178             } catch (InterruptedException e) {
179             }
180             // check if we need to quit, so continue the main loop instead of fallback.
181             continue;
182           }
183           Call call = callsToWrite.poll();
184           if (call.isDone()) {
185             continue;
186           }
187           try {
188             tracedWriteRequest(call);
189           } catch (IOException e) {
190             // exception here means the call has not been added to the pendingCalls yet, so we need
191             // to fail it by our own.
192             if (LOG.isDebugEnabled()) {
193               LOG.debug("call write error for call #" + call.id, e);
194             }
195             call.setException(e);
196             closeConn(e);
197           }
198         }
199       }
200     }
201 
202     /**
203      * Cleans the call not yet sent when we finish.
204      */
205     public void cleanup(IOException e) {
206       IOException ie =
207           new ConnectionClosingException("Connection to " + remoteId.getAddress() + " is closing.");
208       for (Call call : callsToWrite) {
209         call.setException(ie);
210       }
211       callsToWrite.clear();
212     }
213   }
214 
215   BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException {
216     super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
217         rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor,
218         rpcClient.metrics);
219     this.rpcClient = rpcClient;
220     this.connectionHeaderPreamble = getConnectionHeaderPreamble();
221     ConnectionHeader header = getConnectionHeader();
222     ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize());
223     DataOutputStream dos = new DataOutputStream(baos);
224     dos.writeInt(header.getSerializedSize());
225     header.writeTo(dos);
226     assert baos.size() == 4 + header.getSerializedSize();
227     this.connectionHeaderWithLength = baos.getBuffer();
228 
229     UserGroupInformation ticket = remoteId.ticket.getUGI();
230     this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() +
231         ") connection to " + remoteId.getAddress().toString() +
232         ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName()));
233 
234     if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) {
235       callSender = new CallSender(threadName, this.rpcClient.conf);
236       callSender.start();
237     } else {
238       callSender = null;
239     }
240   }
241 
242   // protected for write UT.
243   protected void setupConnection() throws IOException {
244     short ioFailures = 0;
245     short timeoutFailures = 0;
246     while (true) {
247       try {
248         this.socket = this.rpcClient.socketFactory.createSocket();
249         this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay());
250         this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive);
251         if (this.rpcClient.localAddr != null) {
252           this.socket.bind(this.rpcClient.localAddr);
253         }
254         if (this.rpcClient.metrics != null) {
255           this.rpcClient.metrics.incrNsLookups();
256         }
257         InetSocketAddress remoteAddr = remoteId.getAddress().toSocketAddress();
258         if (remoteAddr.isUnresolved()) {
259           if (this.rpcClient.metrics != null) {
260             this.rpcClient.metrics.incrNsLookupsFailed();
261           }
262           throw new UnknownHostException(remoteId.getAddress() + " could not be resolved");
263         }
264         NetUtils.connect(this.socket, remoteAddr, this.rpcClient.connectTO);
265         this.socket.setSoTimeout(this.rpcClient.readTO);
266         return;
267       } catch (SocketTimeoutException toe) {
268         /*
269          * The max number of retries is 45, which amounts to 20s*45 = 15 minutes retries.
270          */
271         handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe);
272       } catch (IOException ie) {
273         handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie);
274       }
275     }
276   }
277 
278   /**
279    * Handle connection failures If the current number of retries is equal to the max number of
280    * retries, stop retrying and throw the exception; Otherwise backoff N seconds and try connecting
281    * again. This Method is only called from inside setupIOstreams(), which is synchronized. Hence
282    * the sleep is synchronized; the locks will be retained.
283    * @param curRetries current number of retries
284    * @param maxRetries max number of retries allowed
285    * @param ioe failure reason
286    * @throws IOException if max number of retries is reached
287    */
288   private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
289       throws IOException {
290     closeSocket();
291 
292     // throw the exception if the maximum number of retries is reached
293     if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
294       throw ioe;
295     }
296 
297     // otherwise back off and retry
298     try {
299       Thread.sleep(this.rpcClient.failureSleep);
300     } catch (InterruptedException ie) {
301       ExceptionUtil.rethrowIfInterrupt(ie);
302     }
303 
304     LOG.info("Retrying connect to server: " + remoteId.getAddress() + " after sleeping " +
305         this.rpcClient.failureSleep + "ms. Already tried " + curRetries + " time(s).");
306   }
307 
308   /*
309    * wait till someone signals us to start reading RPC response or it is idle too long, it is marked
310    * as to be closed, or the client is marked as not running.
311    * @return true if it is time to read a response; false otherwise.
312    */
313   private synchronized boolean waitForWork() {
314     // beware of the concurrent access to the calls list: we can add calls, but as well
315     // remove them.
316     long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose;
317     for (;;) {
318       if (thread == null) {
319         return false;
320       }
321       if (!calls.isEmpty()) {
322         return true;
323       }
324       if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
325         closeConn(
326           new IOException("idle connection closed with " + calls.size() + " pending request(s)"));
327         return false;
328       }
329       try {
330         wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000));
331       } catch (InterruptedException e) {
332       }
333     }
334   }
335 
336   @Override
337   public void run() {
338     if (LOG.isTraceEnabled()) {
339       LOG.trace(threadName + ": starting, connections " + this.rpcClient.connections.size());
340     }
341     while (waitForWork()) {
342       readResponse();
343     }
344     if (LOG.isTraceEnabled()) {
345       LOG.trace(threadName + ": stopped, connections " + this.rpcClient.connections.size());
346     }
347   }
348 
349   private void disposeSasl() {
350     if (saslRpcClient != null) {
351       saslRpcClient.dispose();
352       saslRpcClient = null;
353     }
354   }
355 
356   private boolean setupSaslConnection(final InputStream in2, final OutputStream out2)
357       throws IOException {
358     saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal,
359         this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection",
360           QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
361     return saslRpcClient.saslConnect(in2, out2);
362   }
363 
364   /**
365    * If multiple clients with the same principal try to connect to the same server at the same time,
366    * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to
367    * work around this, what is done is that the client backs off randomly and tries to initiate the
368    * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is
369    * attempted.
370    * <p>
371    * The retry logic is governed by the {@link #shouldAuthenticateOverKrb} method. In case when the
372    * user doesn't have valid credentials, we don't need to retry (from cache or ticket). In such
373    * cases, it is prudent to throw a runtime exception when we receive a SaslException from the
374    * underlying authentication implementation, so there is no retry from other high level (for eg,
375    * HCM or HBaseAdmin).
376    * </p>
377    */
378   private void handleSaslConnectionFailure(final int currRetries, final int maxRetries,
379       final Exception ex, final UserGroupInformation user)
380       throws IOException, InterruptedException {
381     closeSocket();
382     user.doAs(new PrivilegedExceptionAction<Object>() {
383       @Override
384       public Object run() throws IOException, InterruptedException {
385         if (shouldAuthenticateOverKrb()) {
386           if (currRetries < maxRetries) {
387             if (LOG.isDebugEnabled()) {
388               LOG.debug("Exception encountered while connecting to " + "the server : " + ex);
389             }
390             // try re-login
391             relogin();
392             disposeSasl();
393             // have granularity of milliseconds
394             // we are sleeping with the Connection lock held but since this
395             // connection instance is being used for connecting to the server
396             // in question, it is okay
397             Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1);
398             return null;
399           } else {
400             String msg = "Couldn't setup connection for " +
401                 UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
402             LOG.warn(msg, ex);
403             throw new IOException(msg, ex);
404           }
405         } else {
406           LOG.warn("Exception encountered while connecting to " + "the server : " + ex);
407         }
408         if (ex instanceof RemoteException) {
409           throw (RemoteException) ex;
410         }
411         if (ex instanceof SaslException) {
412           String msg = "SASL authentication failed." +
413               " The most likely cause is missing or invalid credentials." + " Consider 'kinit'.";
414           LOG.fatal(msg, ex);
415           throw new RuntimeException(msg, ex);
416         }
417         throw new IOException(ex);
418       }
419     });
420   }
421 
422   private void setupIOstreams() throws IOException {
423     if (socket != null) {
424       // The connection is already available. Perfect.
425       return;
426     }
427 
428     if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) {
429       if (LOG.isDebugEnabled()) {
430         LOG.debug("Not trying to connect to " + remoteId.getAddress() +
431             " this server is in the failed servers list");
432       }
433       throw new FailedServerException(
434           "This server is in the failed servers list: " + remoteId.getAddress());
435     }
436 
437     try {
438       if (LOG.isDebugEnabled()) {
439         LOG.debug("Connecting to " + remoteId.getAddress());
440       }
441 
442       short numRetries = 0;
443       int reloginMaxRetries = this.rpcClient.conf.getInt("hbase.security.relogin.maxretries", 5);
444       while (true) {
445         setupConnection();
446         InputStream inStream = NetUtils.getInputStream(socket);
447         // This creates a socket with a write timeout. This timeout cannot be changed.
448         OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO);
449         // Write out the preamble -- MAGIC, version, and auth to use.
450         writeConnectionHeaderPreamble(outStream);
451         if (useSasl) {
452           final InputStream in2 = inStream;
453           final OutputStream out2 = outStream;
454           UserGroupInformation ticket = getUGI();
455           boolean continueSasl;
456           if (ticket == null) {
457             throw new FatalConnectionException("ticket/user is null");
458           }
459           try {
460             continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
461               @Override
462               public Boolean run() throws IOException {
463                 return setupSaslConnection(in2, out2);
464               }
465             });
466           } catch (Exception ex) {
467             ExceptionUtil.rethrowIfInterrupt(ex);
468             handleSaslConnectionFailure(numRetries++, reloginMaxRetries, ex, ticket);
469             continue;
470           }
471           if (continueSasl) {
472             // Sasl connect is successful. Let's set up Sasl i/o streams.
473             inStream = saslRpcClient.getInputStream(inStream);
474             outStream = saslRpcClient.getOutputStream(outStream);
475           } else {
476             // fall back to simple auth because server told us so.
477             // do not change authMethod and useSasl here, we should start from secure when
478             // reconnecting because regionserver may change its sasl config after restart.
479           }
480         }
481         this.in = new DataInputStream(new BufferedInputStream(inStream));
482         this.out = new DataOutputStream(new BufferedOutputStream(outStream));
483         // Now write out the connection header
484         writeConnectionHeader();
485         break;
486       }
487     } catch (Throwable t) {
488       closeSocket();
489       IOException e = ExceptionUtil.asInterrupt(t);
490       if (e == null) {
491         this.rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), t);
492         if (t instanceof LinkageError) {
493           // probably the hbase hadoop version does not match the running hadoop version
494           e = new DoNotRetryIOException(t);
495         } else if (t instanceof IOException) {
496           e = (IOException) t;
497         } else {
498           e = new IOException("Could not set up IO Streams to " + remoteId.getAddress(), t);
499         }
500       }
501       throw e;
502     }
503 
504     // start the receiver thread after the socket connection has been set up
505     thread = new Thread(this, threadName);
506     thread.setDaemon(true);
507     thread.start();
508   }
509 
510   /**
511    * Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}
512    */
513   private void writeConnectionHeaderPreamble(OutputStream out) throws IOException {
514     out.write(connectionHeaderPreamble);
515     out.flush();
516   }
517 
518   /**
519    * Write the connection header.
520    */
521   private void writeConnectionHeader() throws IOException {
522     this.out.write(connectionHeaderWithLength);
523     this.out.flush();
524   }
525 
526   private void tracedWriteRequest(Call call) throws IOException {
527     try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", call.span)) {
528       writeRequest(call);
529     }
530   }
531 
532   /**
533    * Initiates a call by sending the parameter to the remote server. Note: this is not called from
534    * the Connection thread, but by other threads.
535    * @see #readResponse()
536    */
537   private void writeRequest(Call call) throws IOException {
538     ByteBuffer cellBlock =
539         this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec, this.compressor, call.cells);
540     CellBlockMeta cellBlockMeta;
541     if (cellBlock != null) {
542       cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.limit()).build();
543     } else {
544       cellBlockMeta = null;
545     }
546     RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta);
547 
548     setupIOstreams();
549 
550     // Now we're going to write the call. We take the lock, then check that the connection
551     // is still valid, and, if so we do the write to the socket. If the write fails, we don't
552     // know where we stand, we have to close the connection.
553     if (Thread.interrupted()) {
554       throw new InterruptedIOException();
555     }
556 
557     calls.put(call.id, call); // We put first as we don't want the connection to become idle.
558     // from here, we do not throw any exception to upper layer as the call has been tracked in the
559     // pending calls map.
560     try {
561       call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock));
562     } catch (Throwable t) {
563       if(LOG.isTraceEnabled()) {
564         LOG.trace("Error while writing call, call_id:" + call.id, t);
565       }
566       IOException e = IPCUtil.toIOE(t);
567       closeConn(e);
568       return;
569     }
570     notifyAll();
571   }
572 
573   /*
574    * Receive a response. Because only one receiver, so no synchronization on in.
575    */
576   private void readResponse() {
577     Call call = null;
578     boolean expectedCall = false;
579     try {
580       // See HBaseServer.Call.setResponse for where we write out the response.
581       // Total size of the response. Unused. But have to read it in anyways.
582       int totalSize = in.readInt();
583 
584       // Read the header
585       ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
586       int id = responseHeader.getCallId();
587       call = calls.remove(id); // call.done have to be set before leaving this method
588       expectedCall = (call != null && !call.isDone());
589       if (!expectedCall) {
590         // So we got a response for which we have no corresponding 'call' here on the client-side.
591         // We probably timed out waiting, cleaned up all references, and now the server decides
592         // to return a response. There is nothing we can do w/ the response at this stage. Clean
593         // out the wire of the response so its out of the way and we can get other responses on
594         // this connection.
595         int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader);
596         int whatIsLeftToRead = totalSize - readSoFar;
597         IOUtils.skipFully(in, whatIsLeftToRead);
598         if (call != null) {
599           call.callStats.setResponseSizeBytes(totalSize);
600           call.callStats
601               .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
602         }
603         return;
604       }
605       if (responseHeader.hasException()) {
606         ExceptionResponse exceptionResponse = responseHeader.getException();
607         RemoteException re = createRemoteException(exceptionResponse);
608         call.setException(re);
609         call.callStats.setResponseSizeBytes(totalSize);
610         call.callStats
611             .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
612         if (isFatalConnectionException(exceptionResponse)) {
613           synchronized (this) {
614             closeConn(re);
615           }
616         }
617       } else {
618         Message value = null;
619         if (call.responseDefaultType != null) {
620           Builder builder = call.responseDefaultType.newBuilderForType();
621           ProtobufUtil.mergeDelimitedFrom(builder, in);
622           value = builder.build();
623         }
624         CellScanner cellBlockScanner = null;
625         if (responseHeader.hasCellBlockMeta()) {
626           int size = responseHeader.getCellBlockMeta().getLength();
627           byte[] cellBlock = new byte[size];
628           IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
629           cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec,
630             this.compressor, cellBlock);
631         }
632         call.setResponse(value, cellBlockScanner);
633         call.callStats.setResponseSizeBytes(totalSize);
634         call.callStats
635             .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
636       }
637     } catch (IOException e) {
638       if (expectedCall) {
639         call.setException(e);
640       }
641       if (e instanceof SocketTimeoutException) {
642         // Clean up open calls but don't treat this as a fatal condition,
643         // since we expect certain responses to not make it by the specified
644         // {@link ConnectionId#rpcTimeout}.
645         if (LOG.isTraceEnabled()) {
646           LOG.trace("ignored", e);
647         }
648       } else {
649         synchronized (this) {
650           closeConn(e);
651         }
652       }
653     }
654   }
655 
656   @Override
657   protected synchronized void callTimeout(Call call) {
658     // call sender
659     calls.remove(call.id);
660   }
661 
662   // just close socket input and output.
663   private void closeSocket() {
664     IOUtils.closeStream(out);
665     IOUtils.closeStream(in);
666     IOUtils.closeSocket(socket);
667     out = null;
668     in = null;
669     socket = null;
670   }
671 
672   // close socket, reader, and clean up all pending calls.
673   private void closeConn(IOException e) {
674     if (thread == null) {
675       return;
676     }
677     thread.interrupt();
678     thread = null;
679     closeSocket();
680     if (callSender != null) {
681       callSender.cleanup(e);
682     }
683     for (Call call : calls.values()) {
684       call.setException(e);
685     }
686     calls.clear();
687   }
688 
689   // release all resources, the connection will not be used any more.
690   @Override
691   public synchronized void shutdown() {
692     closed = true;
693     if (callSender != null) {
694       callSender.interrupt();
695     }
696     closeConn(new IOException("connection to " + remoteId.getAddress() + " closed"));
697   }
698 
699   @Override
700   public void cleanupConnection() {
701     // do nothing
702   }
703 
704   @Override
705   public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
706       throws IOException {
707     pcrc.notifyOnCancel(new RpcCallback<Object>() {
708 
709       @Override
710       public void run(Object parameter) {
711         setCancelled(call);
712         synchronized (BlockingRpcConnection.this) {
713           if (callSender != null) {
714             callSender.remove(call);
715           } else {
716             calls.remove(call.id);
717           }
718         }
719       }
720     }, new CancellationCallback() {
721 
722       @Override
723       public void run(boolean cancelled) throws IOException {
724         if (cancelled) {
725           setCancelled(call);
726           return;
727         }
728         scheduleTimeoutTask(call);
729         if (callSender != null) {
730           callSender.sendCall(call);
731         } else {
732           tracedWriteRequest(call);
733         }
734       }
735     });
736   }
737 
738   @Override
739   public synchronized boolean isActive() {
740     return thread != null;
741   }
742 }