1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.ipc;
21
22 import com.google.protobuf.Descriptors.MethodDescriptor;
23 import com.google.protobuf.Message;
24 import com.google.protobuf.Message.Builder;
25 import com.google.protobuf.RpcCallback;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.CellScanner;
28 import org.apache.hadoop.hbase.DoNotRetryIOException;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.ServerName;
31 import org.apache.hadoop.hbase.classification.InterfaceAudience;
32 import org.apache.hadoop.hbase.codec.Codec;
33 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
34 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
35 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
36 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
37 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
38 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
39 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
40 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
41 import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
42 import org.apache.hadoop.hbase.security.AuthMethod;
43 import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
44 import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
45 import org.apache.hadoop.hbase.security.SecurityInfo;
46 import org.apache.hadoop.hbase.security.User;
47 import org.apache.hadoop.hbase.security.UserProvider;
48 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
49 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
50 import org.apache.hadoop.hbase.util.ExceptionUtil;
51 import org.apache.hadoop.hbase.util.Pair;
52 import org.apache.hadoop.hbase.util.PoolMap;
53 import org.apache.hadoop.io.IOUtils;
54 import org.apache.hadoop.io.Text;
55 import org.apache.hadoop.io.compress.CompressionCodec;
56 import org.apache.hadoop.ipc.RemoteException;
57 import org.apache.hadoop.net.NetUtils;
58 import org.apache.hadoop.security.SecurityUtil;
59 import org.apache.hadoop.security.UserGroupInformation;
60 import org.apache.hadoop.security.token.Token;
61 import org.apache.hadoop.security.token.TokenIdentifier;
62 import org.apache.hadoop.security.token.TokenSelector;
63 import org.apache.htrace.Span;
64 import org.apache.htrace.Trace;
65 import org.apache.htrace.TraceScope;
66
67 import javax.net.SocketFactory;
68 import javax.security.sasl.SaslException;
69 import java.io.BufferedInputStream;
70 import java.io.BufferedOutputStream;
71 import java.io.Closeable;
72 import java.io.DataInputStream;
73 import java.io.DataOutputStream;
74 import java.io.IOException;
75 import java.io.InputStream;
76 import java.io.InterruptedIOException;
77 import java.io.OutputStream;
78 import java.net.ConnectException;
79 import java.net.InetSocketAddress;
80 import java.net.Socket;
81 import java.net.SocketAddress;
82 import java.net.SocketTimeoutException;
83 import java.net.UnknownHostException;
84 import java.nio.ByteBuffer;
85 import java.security.PrivilegedExceptionAction;
86 import java.util.HashMap;
87 import java.util.Iterator;
88 import java.util.Map;
89 import java.util.Map.Entry;
90 import java.util.Random;
91 import java.util.concurrent.ArrayBlockingQueue;
92 import java.util.concurrent.BlockingQueue;
93 import java.util.concurrent.ConcurrentSkipListMap;
94 import java.util.concurrent.atomic.AtomicBoolean;
95 import java.util.concurrent.atomic.AtomicInteger;
96
97
98
99
100
101 @InterfaceAudience.Private
102 public class RpcClientImpl extends AbstractRpcClient {
103 protected final AtomicInteger callIdCnt = new AtomicInteger();
104
105 protected final PoolMap<ConnectionId, Connection> connections;
106
107 protected final AtomicBoolean running = new AtomicBoolean(true);
108
109 protected final FailedServers failedServers;
110
111 protected final SocketFactory socketFactory;
112
113 protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
114 TokenSelector<? extends TokenIdentifier>> tokenHandlers =
115 new HashMap<AuthenticationProtos.TokenIdentifier.Kind,
116 TokenSelector<? extends TokenIdentifier>>();
117 static {
118 tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
119 new AuthenticationTokenSelector());
120 }
121
122
123
124
125
126 protected Connection createConnection(ConnectionId remoteId, final Codec codec,
127 final CompressionCodec compressor)
128 throws IOException {
129 return new Connection(remoteId, codec, compressor);
130 }
131
132
133
134
135 private static class CallFuture {
136 final Call call;
137 final int priority;
138 final Span span;
139
140
141 final static CallFuture DEATH_PILL = new CallFuture(null, -1, null);
142
143 CallFuture(Call call, int priority, Span span) {
144 this.call = call;
145 this.priority = priority;
146 this.span = span;
147 }
148 }
149
150
151
152
153 protected class Connection extends Thread {
154 private ConnectionHeader header;
155 protected ConnectionId remoteId;
156 protected Socket socket = null;
157 protected DataInputStream in;
158 protected DataOutputStream out;
159 private Object outLock = new Object();
160 private InetSocketAddress server;
161 private String serverPrincipal;
162 private AuthMethod authMethod;
163 private boolean useSasl;
164 private Token<? extends TokenIdentifier> token;
165 private HBaseSaslRpcClient saslRpcClient;
166 private int reloginMaxBackoff;
167 private final Codec codec;
168 private final CompressionCodec compressor;
169
170
171 protected final ConcurrentSkipListMap<Integer, Call> calls =
172 new ConcurrentSkipListMap<Integer, Call>();
173
174 protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();
175 protected final CallSender callSender;
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196 private class CallSender extends Thread implements Closeable {
197 protected final BlockingQueue<CallFuture> callsToWrite;
198
199
200 public CallFuture sendCall(Call call, int priority, Span span)
201 throws InterruptedException, IOException {
202 CallFuture cts = new CallFuture(call, priority, span);
203 if (!callsToWrite.offer(cts)) {
204 throw new IOException("Can't add the call " + call.id +
205 " to the write queue. callsToWrite.size()=" + callsToWrite.size());
206 }
207 checkIsOpen();
208
209 return cts;
210 }
211
212 @Override
213 public void close(){
214 assert shouldCloseConnection.get();
215 callsToWrite.offer(CallFuture.DEATH_PILL);
216
217
218 }
219
220 CallSender(String name, Configuration conf) {
221 int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
222 callsToWrite = new ArrayBlockingQueue<CallFuture>(queueSize);
223 setDaemon(true);
224 setName(name + " - writer");
225 }
226
227 public void remove(CallFuture cts){
228 callsToWrite.remove(cts);
229
230
231
232 calls.remove(cts.call.id);
233 cts.call.callComplete();
234 }
235
236
237
238
239 @Override
240 public void run() {
241 while (!shouldCloseConnection.get()) {
242 CallFuture cts = null;
243 try {
244 cts = callsToWrite.take();
245 } catch (InterruptedException e) {
246 markClosed(new InterruptedIOException());
247 }
248
249 if (cts == null || cts == CallFuture.DEATH_PILL) {
250 assert shouldCloseConnection.get();
251 break;
252 }
253
254 if (cts.call.done) {
255 continue;
256 }
257
258 if (cts.call.checkAndSetTimeout()) {
259 continue;
260 }
261
262 try {
263 Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span);
264 } catch (IOException e) {
265 LOG.warn("call write error for call #" + cts.call.id + ", message =" + e.getMessage());
266 cts.call.setException(e);
267 markClosed(e);
268 }
269 }
270
271 cleanup();
272 }
273
274
275
276
277 private void cleanup() {
278 assert shouldCloseConnection.get();
279
280 IOException ie = new ConnectionClosingException("Connection to " + server + " is closing.");
281 while (true) {
282 CallFuture cts = callsToWrite.poll();
283 if (cts == null) {
284 break;
285 }
286 if (cts.call != null && !cts.call.done) {
287 cts.call.setException(ie);
288 }
289 }
290 }
291 }
292
293 Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
294 throws IOException {
295 if (remoteId.getAddress().isUnresolved()) {
296 throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
297 }
298 this.server = remoteId.getAddress();
299 this.codec = codec;
300 this.compressor = compressor;
301
302 UserGroupInformation ticket = remoteId.getTicket().getUGI();
303 SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
304 this.useSasl = userProvider.isHBaseSecurityEnabled();
305 if (useSasl && securityInfo != null) {
306 AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
307 if (tokenKind != null) {
308 TokenSelector<? extends TokenIdentifier> tokenSelector =
309 tokenHandlers.get(tokenKind);
310 if (tokenSelector != null) {
311 token = tokenSelector.selectToken(new Text(clusterId),
312 ticket.getTokens());
313 } else if (LOG.isDebugEnabled()) {
314 LOG.debug("No token selector found for type "+tokenKind);
315 }
316 }
317 String serverKey = securityInfo.getServerPrincipal();
318 if (serverKey == null) {
319 throw new IOException(
320 "Can't obtain server Kerberos config key from SecurityInfo");
321 }
322 serverPrincipal = SecurityUtil.getServerPrincipal(
323 conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
324 if (LOG.isDebugEnabled()) {
325 LOG.debug("RPC Server Kerberos principal name for service="
326 + remoteId.getServiceName() + " is " + serverPrincipal);
327 }
328 }
329
330 if (!useSasl) {
331 authMethod = AuthMethod.SIMPLE;
332 } else if (token != null) {
333 authMethod = AuthMethod.DIGEST;
334 } else {
335 authMethod = AuthMethod.KERBEROS;
336 }
337
338 if (LOG.isDebugEnabled()) {
339 LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
340 ", sasl=" + useSasl);
341 }
342 reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
343 this.remoteId = remoteId;
344
345 ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
346 builder.setServiceName(remoteId.getServiceName());
347 UserInformation userInfoPB = getUserInfo(ticket);
348 if (userInfoPB != null) {
349 builder.setUserInfo(userInfoPB);
350 }
351 if (this.codec != null) {
352 builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
353 }
354 if (this.compressor != null) {
355 builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
356 }
357 this.header = builder.build();
358
359 this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
360 remoteId.getAddress().toString() +
361 ((ticket==null)?" from an unknown user": (" from "
362 + ticket.getUserName())));
363 this.setDaemon(true);
364
365 if (conf.getBoolean(SPECIFIC_WRITE_THREAD, false)) {
366 callSender = new CallSender(getName(), conf);
367 callSender.start();
368 } else {
369 callSender = null;
370 }
371 }
372
373 private UserInformation getUserInfo(UserGroupInformation ugi) {
374 if (ugi == null || authMethod == AuthMethod.DIGEST) {
375
376 return null;
377 }
378 UserInformation.Builder userInfoPB = UserInformation.newBuilder();
379 if (authMethod == AuthMethod.KERBEROS) {
380
381 userInfoPB.setEffectiveUser(ugi.getUserName());
382 } else if (authMethod == AuthMethod.SIMPLE) {
383
384 userInfoPB.setEffectiveUser(ugi.getUserName());
385 if (ugi.getRealUser() != null) {
386 userInfoPB.setRealUser(ugi.getRealUser().getUserName());
387 }
388 }
389 return userInfoPB.build();
390 }
391
392 protected synchronized void setupConnection() throws IOException {
393 short ioFailures = 0;
394 short timeoutFailures = 0;
395 while (true) {
396 try {
397 this.socket = socketFactory.createSocket();
398 this.socket.setTcpNoDelay(tcpNoDelay);
399 this.socket.setKeepAlive(tcpKeepAlive);
400 if (localAddr != null) {
401 this.socket.bind(localAddr);
402 }
403 NetUtils.connect(this.socket, remoteId.getAddress(), connectTO);
404 this.socket.setSoTimeout(readTO);
405 return;
406 } catch (SocketTimeoutException toe) {
407
408
409
410 handleConnectionFailure(timeoutFailures++, maxRetries, toe);
411 } catch (IOException ie) {
412 handleConnectionFailure(ioFailures++, maxRetries, ie);
413 }
414 }
415 }
416
417 protected synchronized void closeConnection() {
418 if (socket == null) {
419 return;
420 }
421
422
423 try {
424 if (socket.getOutputStream() != null) {
425 socket.getOutputStream().close();
426 }
427 } catch (IOException ignored) {
428 if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
429 }
430 try {
431 if (socket.getInputStream() != null) {
432 socket.getInputStream().close();
433 }
434 } catch (IOException ignored) {
435 if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
436 }
437 try {
438 if (socket.getChannel() != null) {
439 socket.getChannel().close();
440 }
441 } catch (IOException ignored) {
442 if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
443 }
444 try {
445 socket.close();
446 } catch (IOException e) {
447 LOG.warn("Not able to close a socket", e);
448 }
449
450
451
452 socket = null;
453 }
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470 private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
471 throws IOException {
472 closeConnection();
473
474
475 if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
476 throw ioe;
477 }
478
479
480 try {
481 Thread.sleep(failureSleep);
482 } catch (InterruptedException ie) {
483 ExceptionUtil.rethrowIfInterrupt(ie);
484 }
485
486 LOG.info("Retrying connect to server: " + remoteId.getAddress() +
487 " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
488 " time(s).");
489 }
490
491
492
493
494 private void checkIsOpen() throws IOException {
495 if (shouldCloseConnection.get()) {
496 throw new ConnectionClosingException(getName() + " is closing");
497 }
498 }
499
500
501
502
503
504
505
506 protected synchronized boolean waitForWork() throws InterruptedException {
507
508
509 long waitUntil = EnvironmentEdgeManager.currentTime() + minIdleTimeBeforeClose;
510
511 while (true) {
512 if (shouldCloseConnection.get()) {
513 return false;
514 }
515
516 if (!running.get()) {
517 markClosed(new IOException("stopped with " + calls.size() + " pending request(s)"));
518 return false;
519 }
520
521 if (!calls.isEmpty()) {
522
523
524 return true;
525 }
526
527 if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
528
529
530
531
532 markClosed(new IOException(
533 "idle connection closed with " + calls.size() + " pending request(s)"));
534 return false;
535 }
536
537 wait(Math.min(minIdleTimeBeforeClose, 1000));
538 }
539 }
540
541 public InetSocketAddress getRemoteAddress() {
542 return remoteId.getAddress();
543 }
544
545 @Override
546 public void run() {
547 if (LOG.isTraceEnabled()) {
548 LOG.trace(getName() + ": starting, connections " + connections.size());
549 }
550
551 try {
552 while (waitForWork()) {
553 readResponse();
554 }
555 } catch (InterruptedException t) {
556 if (LOG.isTraceEnabled()) {
557 LOG.trace(getName() + ": interrupted while waiting for call responses");
558 }
559 markClosed(ExceptionUtil.asInterrupt(t));
560 } catch (Throwable t) {
561 if (LOG.isDebugEnabled()) {
562 LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t);
563 }
564 markClosed(new IOException("Unexpected throwable while waiting call responses", t));
565 }
566
567 close();
568
569 if (LOG.isTraceEnabled()) {
570 LOG.trace(getName() + ": stopped, connections " + connections.size());
571 }
572 }
573
574 private synchronized void disposeSasl() {
575 if (saslRpcClient != null) {
576 try {
577 saslRpcClient.dispose();
578 saslRpcClient = null;
579 } catch (IOException ioe) {
580 LOG.error("Error disposing of SASL client", ioe);
581 }
582 }
583 }
584
585 private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
586 UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
587 UserGroupInformation currentUser =
588 UserGroupInformation.getCurrentUser();
589 UserGroupInformation realUser = currentUser.getRealUser();
590 return authMethod == AuthMethod.KERBEROS &&
591 loginUser != null &&
592
593 loginUser.hasKerberosCredentials() &&
594
595
596 (loginUser.equals(currentUser) || loginUser.equals(realUser));
597 }
598
599 private synchronized boolean setupSaslConnection(final InputStream in2,
600 final OutputStream out2) throws IOException {
601 saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
602 conf.get("hbase.rpc.protection",
603 QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
604 return saslRpcClient.saslConnect(in2, out2);
605 }
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625 private synchronized void handleSaslConnectionFailure(
626 final int currRetries,
627 final int maxRetries, final Exception ex, final Random rand,
628 final UserGroupInformation user)
629 throws IOException, InterruptedException{
630 user.doAs(new PrivilegedExceptionAction<Object>() {
631 @Override
632 public Object run() throws IOException, InterruptedException {
633 closeConnection();
634 if (shouldAuthenticateOverKrb()) {
635 if (currRetries < maxRetries) {
636 if (LOG.isDebugEnabled()) {
637 LOG.debug("Exception encountered while connecting to " +
638 "the server : " + ex);
639 }
640
641 if (UserGroupInformation.isLoginKeytabBased()) {
642 UserGroupInformation.getLoginUser().reloginFromKeytab();
643 } else {
644 UserGroupInformation.getLoginUser().reloginFromTicketCache();
645 }
646 disposeSasl();
647
648
649
650
651 Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
652 return null;
653 } else {
654 String msg = "Couldn't setup connection for " +
655 UserGroupInformation.getLoginUser().getUserName() +
656 " to " + serverPrincipal;
657 LOG.warn(msg);
658 throw (IOException) new IOException(msg).initCause(ex);
659 }
660 } else {
661 LOG.warn("Exception encountered while connecting to " +
662 "the server : " + ex);
663 }
664 if (ex instanceof RemoteException) {
665 throw (RemoteException)ex;
666 }
667 if (ex instanceof SaslException) {
668 String msg = "SASL authentication failed." +
669 " The most likely cause is missing or invalid credentials." +
670 " Consider 'kinit'.";
671 LOG.fatal(msg, ex);
672 throw new RuntimeException(msg, ex);
673 }
674 throw new IOException(ex);
675 }
676 });
677 }
678
679 protected synchronized void setupIOstreams() throws IOException {
680 if (socket != null) {
681
682 return;
683 }
684
685 if (shouldCloseConnection.get()){
686 throw new ConnectionClosingException("This connection is closing");
687 }
688
689 if (failedServers.isFailedServer(remoteId.getAddress())) {
690 if (LOG.isDebugEnabled()) {
691 LOG.debug("Not trying to connect to " + server +
692 " this server is in the failed servers list");
693 }
694 IOException e = new FailedServerException(
695 "This server is in the failed servers list: " + server);
696 markClosed(e);
697 close();
698 throw e;
699 }
700
701 try {
702 if (LOG.isDebugEnabled()) {
703 LOG.debug("Connecting to " + server);
704 }
705 short numRetries = 0;
706 final short MAX_RETRIES = 5;
707 Random rand = null;
708 while (true) {
709 setupConnection();
710 InputStream inStream = NetUtils.getInputStream(socket);
711
712 OutputStream outStream = NetUtils.getOutputStream(socket, writeTO);
713
714 writeConnectionHeaderPreamble(outStream);
715 if (useSasl) {
716 final InputStream in2 = inStream;
717 final OutputStream out2 = outStream;
718 UserGroupInformation ticket = remoteId.getTicket().getUGI();
719 if (authMethod == AuthMethod.KERBEROS) {
720 if (ticket != null && ticket.getRealUser() != null) {
721 ticket = ticket.getRealUser();
722 }
723 }
724 boolean continueSasl;
725 if (ticket == null) throw new FatalConnectionException("ticket/user is null");
726 try {
727 continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
728 @Override
729 public Boolean run() throws IOException {
730 return setupSaslConnection(in2, out2);
731 }
732 });
733 } catch (Exception ex) {
734 ExceptionUtil.rethrowIfInterrupt(ex);
735 if (rand == null) {
736 rand = new Random();
737 }
738 handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
739 continue;
740 }
741 if (continueSasl) {
742
743 inStream = saslRpcClient.getInputStream(inStream);
744 outStream = saslRpcClient.getOutputStream(outStream);
745 } else {
746
747 authMethod = AuthMethod.SIMPLE;
748 useSasl = false;
749 }
750 }
751 this.in = new DataInputStream(new BufferedInputStream(inStream));
752 synchronized (this.outLock) {
753 this.out = new DataOutputStream(new BufferedOutputStream(outStream));
754 }
755
756 writeConnectionHeader();
757
758
759 start();
760 return;
761 }
762 } catch (Throwable t) {
763 IOException e = ExceptionUtil.asInterrupt(t);
764 if (e == null) {
765 failedServers.addToFailedServers(remoteId.address);
766 if (t instanceof LinkageError) {
767
768 e = new DoNotRetryIOException(t);
769 } else if (t instanceof IOException) {
770 e = (IOException) t;
771 } else {
772 e = new IOException("Could not set up IO Streams to " + server, t);
773 }
774 }
775 markClosed(e);
776 close();
777 throw e;
778 }
779 }
780
781
782
783
784 private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
785
786
787
788
789
790 int rpcHeaderLen = HConstants.RPC_HEADER.array().length;
791 byte [] preamble = new byte [rpcHeaderLen + 2];
792 System.arraycopy(HConstants.RPC_HEADER.array(), 0, preamble, 0, rpcHeaderLen);
793 preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
794 preamble[rpcHeaderLen + 1] = authMethod.code;
795 outStream.write(preamble);
796 outStream.flush();
797 }
798
799
800
801
802 private synchronized void writeConnectionHeader() throws IOException {
803 synchronized (this.outLock) {
804 this.out.writeInt(this.header.getSerializedSize());
805 this.header.writeTo(this.out);
806 this.out.flush();
807 }
808 }
809
810
811 protected synchronized void close() {
812 if (!shouldCloseConnection.get()) {
813 LOG.error(getName() + ": the connection is not in the closed state");
814 return;
815 }
816
817
818
819 synchronized (connections) {
820 connections.removeValue(remoteId, this);
821 }
822
823
824 synchronized(this.outLock) {
825 if (this.out != null) {
826 IOUtils.closeStream(out);
827 this.out = null;
828 }
829 }
830 IOUtils.closeStream(in);
831 this.in = null;
832 disposeSasl();
833
834
835 if (LOG.isTraceEnabled()) {
836 LOG.trace(getName() + ": closing ipc connection to " + server);
837 }
838
839 cleanupCalls(true);
840
841 if (LOG.isTraceEnabled()) {
842 LOG.trace(getName() + ": ipc connection to " + server + " closed");
843 }
844 }
845
846 protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException {
847 TraceScope ts = Trace.continueSpan(span);
848 try {
849 writeRequest(call, priority, span);
850 } finally {
851 ts.close();
852 }
853 }
854
855
856
857
858
859
860
861 private void writeRequest(Call call, final int priority, Span span) throws IOException {
862 RequestHeader.Builder builder = RequestHeader.newBuilder();
863 builder.setCallId(call.id);
864 if (span != null) {
865 builder.setTraceInfo(
866 RPCTInfo.newBuilder().setParentId(span.getSpanId()).setTraceId(span.getTraceId()));
867 }
868 builder.setMethodName(call.md.getName());
869 builder.setRequestParam(call.param != null);
870 ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells);
871 if (cellBlock != null) {
872 CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
873 cellBlockBuilder.setLength(cellBlock.limit());
874 builder.setCellBlockMeta(cellBlockBuilder.build());
875 }
876
877 if (priority != 0) builder.setPriority(priority);
878 RequestHeader header = builder.build();
879
880 setupIOstreams();
881
882
883
884
885 checkIsOpen();
886 IOException writeException = null;
887 synchronized (this.outLock) {
888 if (Thread.interrupted()) throw new InterruptedIOException();
889
890 calls.put(call.id, call);
891 checkIsOpen();
892
893 try {
894 IPCUtil.write(this.out, header, call.param, cellBlock);
895 } catch (IOException e) {
896
897
898 shouldCloseConnection.set(true);
899 writeException = e;
900 interrupt();
901 }
902 }
903
904
905
906 synchronized (this) {
907 notifyAll();
908 }
909
910
911 if (writeException != null) throw writeException;
912 }
913
914
915
916
917 protected void readResponse() {
918 if (shouldCloseConnection.get()) return;
919 Call call = null;
920 boolean expectedCall = false;
921 try {
922
923
924 int totalSize = in.readInt();
925
926
927 ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
928 int id = responseHeader.getCallId();
929 call = calls.remove(id);
930 expectedCall = (call != null && !call.done);
931 if (!expectedCall) {
932
933
934
935
936
937 int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
938 int whatIsLeftToRead = totalSize - readSoFar;
939 IOUtils.skipFully(in, whatIsLeftToRead);
940 return;
941 }
942 if (responseHeader.hasException()) {
943 ExceptionResponse exceptionResponse = responseHeader.getException();
944 RemoteException re = createRemoteException(exceptionResponse);
945 call.setException(re);
946 if (isFatalConnectionException(exceptionResponse)) {
947 markClosed(re);
948 }
949 } else {
950 Message value = null;
951 if (call.responseDefaultType != null) {
952 Builder builder = call.responseDefaultType.newBuilderForType();
953 builder.mergeDelimitedFrom(in);
954 value = builder.build();
955 }
956 CellScanner cellBlockScanner = null;
957 if (responseHeader.hasCellBlockMeta()) {
958 int size = responseHeader.getCellBlockMeta().getLength();
959 byte [] cellBlock = new byte[size];
960 IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
961 cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock);
962 }
963 call.setResponse(value, cellBlockScanner);
964 }
965 } catch (IOException e) {
966 if (expectedCall) call.setException(e);
967 if (e instanceof SocketTimeoutException) {
968
969
970
971 if (LOG.isTraceEnabled()) LOG.trace("ignored", e);
972 } else {
973
974 markClosed(e);
975 }
976 } finally {
977 cleanupCalls(false);
978 }
979 }
980
981
982
983
984 private boolean isFatalConnectionException(final ExceptionResponse e) {
985 return e.getExceptionClassName().
986 equals(FatalConnectionException.class.getName());
987 }
988
989
990
991
992
993 private RemoteException createRemoteException(final ExceptionResponse e) {
994 String innerExceptionClassName = e.getExceptionClassName();
995 boolean doNotRetry = e.getDoNotRetry();
996 return e.hasHostname()?
997
998 new RemoteWithExtrasException(innerExceptionClassName,
999 e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry):
1000 new RemoteWithExtrasException(innerExceptionClassName,
1001 e.getStackTrace(), doNotRetry);
1002 }
1003
1004 protected synchronized void markClosed(IOException e) {
1005 if (e == null) throw new NullPointerException();
1006
1007 if (shouldCloseConnection.compareAndSet(false, true)) {
1008 if (LOG.isTraceEnabled()) {
1009 LOG.trace(getName() + ": marking at should close, reason: " + e.getMessage());
1010 }
1011 if (callSender != null) {
1012 callSender.close();
1013 }
1014 notifyAll();
1015 }
1016 }
1017
1018
1019
1020
1021
1022
1023 protected synchronized void cleanupCalls(boolean allCalls) {
1024 Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
1025 while (itor.hasNext()) {
1026 Call c = itor.next().getValue();
1027 if (c.done) {
1028
1029 itor.remove();
1030 } else if (allCalls) {
1031 long waitTime = EnvironmentEdgeManager.currentTime() - c.getStartTime();
1032 IOException ie = new ConnectionClosingException("Connection to " + getRemoteAddress()
1033 + " is closing. Call id=" + c.id + ", waitTime=" + waitTime);
1034 c.setException(ie);
1035 itor.remove();
1036 } else if (c.checkAndSetTimeout()) {
1037 itor.remove();
1038 } else {
1039
1040
1041
1042 break;
1043 }
1044 }
1045 }
1046 }
1047
1048
1049
1050
1051
1052
1053
1054 RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory) {
1055 this(conf, clusterId, factory, null);
1056 }
1057
1058
1059
1060
1061
1062
1063
1064
1065 RpcClientImpl(Configuration conf, String clusterId, SocketFactory factory,
1066 SocketAddress localAddr) {
1067 super(conf, clusterId, localAddr);
1068
1069 this.socketFactory = factory;
1070 this.connections = new PoolMap<ConnectionId, Connection>(getPoolType(conf), getPoolSize(conf));
1071 this.failedServers = new FailedServers(conf);
1072 }
1073
1074
1075
1076
1077
1078
1079 public RpcClientImpl(Configuration conf, String clusterId) {
1080 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null);
1081 }
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092 public RpcClientImpl(Configuration conf, String clusterId, SocketAddress localAddr) {
1093 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr);
1094 }
1095
1096
1097
1098 @Override
1099 public void close() {
1100 if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client");
1101 if (!running.compareAndSet(true, false)) return;
1102
1103
1104 synchronized (connections) {
1105 for (Connection conn : connections.values()) {
1106 conn.interrupt();
1107 }
1108 }
1109
1110
1111 while (!connections.isEmpty()) {
1112 try {
1113 Thread.sleep(100);
1114 } catch (InterruptedException e) {
1115 LOG.info("Interrupted while stopping the client. We still have " + connections.size() +
1116 " connections.");
1117 Thread.currentThread().interrupt();
1118 return;
1119 }
1120 }
1121 }
1122
1123 Pair<Message, CellScanner> call(PayloadCarryingRpcController pcrc,
1124 MethodDescriptor md, Message param, CellScanner cells,
1125 Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
1126 throws InterruptedException, IOException {
1127 return
1128 call(pcrc, md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
1129 }
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143 @Override
1144 protected Pair<Message, CellScanner> call(PayloadCarryingRpcController pcrc, MethodDescriptor md,
1145 Message param, CellScanner cells,
1146 Message returnType, User ticket, InetSocketAddress addr, int callTimeout, int priority)
1147 throws IOException, InterruptedException {
1148 final Call call = new Call(
1149 this.callIdCnt.getAndIncrement(),
1150 md, param, cells, returnType, callTimeout);
1151
1152 final Connection connection = getConnection(ticket, call, addr, this.codec, this.compressor);
1153
1154 final CallFuture cts;
1155 if (connection.callSender != null) {
1156 cts = connection.callSender.sendCall(call, priority, Trace.currentSpan());
1157 if (pcrc != null) {
1158 pcrc.notifyOnCancel(new RpcCallback<Object>() {
1159 @Override
1160 public void run(Object parameter) {
1161 connection.callSender.remove(cts);
1162 }
1163 });
1164 if (pcrc.isCanceled()) {
1165
1166 call.callComplete();
1167 return new Pair<Message, CellScanner>(call.response, call.cells);
1168 }
1169 }
1170
1171 } else {
1172 cts = null;
1173 connection.tracedWriteRequest(call, priority, Trace.currentSpan());
1174 }
1175
1176 while (!call.done) {
1177 if (call.checkAndSetTimeout()) {
1178 if (cts != null) connection.callSender.remove(cts);
1179 break;
1180 }
1181 if (connection.shouldCloseConnection.get()) {
1182 throw new ConnectionClosingException("Call id=" + call.id +
1183 " on server " + addr + " aborted: connection is closing");
1184 }
1185 try {
1186 synchronized (call) {
1187 if (call.done) break;
1188 call.wait(Math.min(call.remainingTime(), 1000) + 1);
1189 }
1190 } catch (InterruptedException e) {
1191 call.setException(new InterruptedIOException());
1192 if (cts != null) connection.callSender.remove(cts);
1193 throw e;
1194 }
1195 }
1196
1197 if (call.error != null) {
1198 if (call.error instanceof RemoteException) {
1199 call.error.fillInStackTrace();
1200 throw call.error;
1201 }
1202
1203 throw wrapException(addr, call.error);
1204 }
1205
1206 return new Pair<Message, CellScanner>(call.response, call.cells);
1207 }
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222 protected IOException wrapException(InetSocketAddress addr,
1223 IOException exception) {
1224 if (exception instanceof ConnectException) {
1225
1226 return (ConnectException)new ConnectException(
1227 "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
1228 } else if (exception instanceof SocketTimeoutException) {
1229 return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
1230 " failed because " + exception).initCause(exception);
1231 } else if (exception instanceof ConnectionClosingException){
1232 return (ConnectionClosingException) new ConnectionClosingException(
1233 "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
1234 } else {
1235 return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
1236 exception).initCause(exception);
1237 }
1238 }
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248 @Override
1249 public void cancelConnections(ServerName sn) {
1250 synchronized (connections) {
1251 for (Connection connection : connections.values()) {
1252 if (connection.isAlive() &&
1253 connection.getRemoteAddress().getPort() == sn.getPort() &&
1254 connection.getRemoteAddress().getHostName().equals(sn.getHostname())) {
1255 LOG.info("The server on " + sn.toString() +
1256 " is dead - stopping the connection " + connection.remoteId);
1257 connection.interrupt();
1258
1259 }
1260 }
1261 }
1262 }
1263
1264
1265
1266
1267
1268 protected Connection getConnection(User ticket, Call call, InetSocketAddress addr,
1269 final Codec codec, final CompressionCodec compressor)
1270 throws IOException {
1271 if (!running.get()) throw new StoppedRpcClientException();
1272 Connection connection;
1273 ConnectionId remoteId =
1274 new ConnectionId(ticket, call.md.getService().getName(), addr);
1275 synchronized (connections) {
1276 connection = connections.get(remoteId);
1277 if (connection == null) {
1278 connection = createConnection(remoteId, this.codec, this.compressor);
1279 connections.put(remoteId, connection);
1280 }
1281 }
1282
1283 return connection;
1284 }
1285 }