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 java.io.BufferedInputStream;
23 import java.io.BufferedOutputStream;
24 import java.io.DataInputStream;
25 import java.io.DataOutputStream;
26 import java.io.FilterInputStream;
27 import java.io.IOException;
28 import java.io.InputStream;
29 import java.io.OutputStream;
30 import java.net.ConnectException;
31 import java.net.InetSocketAddress;
32 import java.net.Socket;
33 import java.net.SocketAddress;
34 import java.net.SocketException;
35 import java.net.SocketTimeoutException;
36 import java.net.UnknownHostException;
37 import java.nio.ByteBuffer;
38 import java.security.PrivilegedExceptionAction;
39 import java.util.HashMap;
40 import java.util.Iterator;
41 import java.util.LinkedList;
42 import java.util.Map;
43 import java.util.Map.Entry;
44 import java.util.Random;
45 import java.util.concurrent.ConcurrentSkipListMap;
46 import java.util.concurrent.atomic.AtomicBoolean;
47 import java.util.concurrent.atomic.AtomicLong;
48
49 import javax.net.SocketFactory;
50 import javax.security.sasl.SaslException;
51
52 import org.apache.commons.logging.Log;
53 import org.apache.commons.logging.LogFactory;
54 import org.apache.hadoop.classification.InterfaceAudience;
55 import org.apache.hadoop.classification.InterfaceStability;
56 import org.apache.hadoop.conf.Configuration;
57 import org.apache.hadoop.hbase.CellScanner;
58 import org.apache.hadoop.hbase.HBaseIOException;
59 import org.apache.hadoop.hbase.HConstants;
60 import org.apache.hadoop.hbase.ServerName;
61 import org.apache.hadoop.hbase.codec.Codec;
62 import org.apache.hadoop.hbase.codec.KeyValueCodec;
63 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
64 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
65 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
66 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
67 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
68 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
69 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
70 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
71 import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo;
72 import org.apache.hadoop.hbase.security.AuthMethod;
73 import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
74 import org.apache.hadoop.hbase.security.SecurityInfo;
75 import org.apache.hadoop.hbase.security.User;
76 import org.apache.hadoop.hbase.security.UserProvider;
77 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
78 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
79 import org.apache.hadoop.hbase.util.Pair;
80 import org.apache.hadoop.hbase.util.PoolMap;
81 import org.apache.hadoop.hbase.util.PoolMap.PoolType;
82 import org.apache.hadoop.io.IOUtils;
83 import org.apache.hadoop.io.Text;
84 import org.apache.hadoop.io.compress.CompressionCodec;
85 import org.apache.hadoop.ipc.RemoteException;
86 import org.apache.hadoop.net.NetUtils;
87 import org.apache.hadoop.security.SecurityUtil;
88 import org.apache.hadoop.security.UserGroupInformation;
89 import org.apache.hadoop.security.token.Token;
90 import org.apache.hadoop.security.token.TokenIdentifier;
91 import org.apache.hadoop.security.token.TokenSelector;
92 import org.cloudera.htrace.Span;
93 import org.cloudera.htrace.Trace;
94
95 import com.google.common.annotations.VisibleForTesting;
96 import com.google.protobuf.BlockingRpcChannel;
97 import com.google.protobuf.Descriptors.MethodDescriptor;
98 import com.google.protobuf.Message;
99 import com.google.protobuf.Message.Builder;
100 import com.google.protobuf.RpcController;
101 import com.google.protobuf.ServiceException;
102 import com.google.protobuf.TextFormat;
103
104
105
106
107
108
109 @InterfaceAudience.Private
110 public class RpcClient {
111
112
113 public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RpcClient");
114 protected final PoolMap<ConnectionId, Connection> connections;
115
116 protected int counter;
117 protected final AtomicBoolean running = new AtomicBoolean(true);
118 final protected Configuration conf;
119 final protected int maxIdleTime;
120
121 final protected int maxRetries;
122 final protected long failureSleep;
123 protected final boolean tcpNoDelay;
124 protected final boolean tcpKeepAlive;
125 protected int pingInterval;
126 protected FailedServers failedServers;
127 private final Codec codec;
128 private final CompressionCodec compressor;
129 private final IPCUtil ipcUtil;
130
131 protected final SocketFactory socketFactory;
132 protected String clusterId;
133 protected final SocketAddress localAddr;
134
135 private final boolean fallbackAllowed;
136 private UserProvider userProvider;
137
138 final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
139 final private static String SOCKET_TIMEOUT = "ipc.socket.timeout";
140 final static int DEFAULT_PING_INTERVAL = 60000;
141 final static int DEFAULT_SOCKET_TIMEOUT = 20000;
142 final static int PING_CALL_ID = -1;
143
144 public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
145 public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
146
147 public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
148 "hbase.ipc.client.fallback-to-simple-auth-allowed";
149 public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
150
151
152
153
154
155
156 private static ThreadLocal<Integer> rpcTimeout = new ThreadLocal<Integer>() {
157 @Override
158 protected Integer initialValue() {
159 return HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
160 }
161 };
162
163
164
165
166 static class FailedServers {
167 private final LinkedList<Pair<Long, String>> failedServers = new
168 LinkedList<Pair<Long, java.lang.String>>();
169 private final int recheckServersTimeout;
170
171 FailedServers(Configuration conf) {
172 this.recheckServersTimeout = conf.getInt(
173 FAILED_SERVER_EXPIRY_KEY, FAILED_SERVER_EXPIRY_DEFAULT);
174 }
175
176
177
178
179 public synchronized void addToFailedServers(InetSocketAddress address) {
180 final long expiry = EnvironmentEdgeManager.currentTimeMillis() + recheckServersTimeout;
181 failedServers.addFirst(new Pair<Long, String>(expiry, address.toString()));
182 }
183
184
185
186
187
188
189 public synchronized boolean isFailedServer(final InetSocketAddress address) {
190 if (failedServers.isEmpty()) {
191 return false;
192 }
193
194 final String lookup = address.toString();
195 final long now = EnvironmentEdgeManager.currentTimeMillis();
196
197
198 Iterator<Pair<Long, String>> it = failedServers.iterator();
199 while (it.hasNext()) {
200 Pair<Long, String> cur = it.next();
201 if (cur.getFirst() < now) {
202 it.remove();
203 } else {
204 if (lookup.equals(cur.getSecond())) {
205 return true;
206 }
207 }
208 }
209
210 return false;
211 }
212 }
213
214 @SuppressWarnings("serial")
215 @InterfaceAudience.Public
216 @InterfaceStability.Evolving
217
218 public static class FailedServerException extends HBaseIOException {
219 public FailedServerException(String s) {
220 super(s);
221 }
222 }
223
224
225
226
227
228
229
230
231
232 public static void setPingInterval(Configuration conf, int pingInterval) {
233 conf.setInt(PING_INTERVAL_NAME, pingInterval);
234 }
235
236
237
238
239
240
241
242
243 static int getPingInterval(Configuration conf) {
244 return conf.getInt(PING_INTERVAL_NAME, DEFAULT_PING_INTERVAL);
245 }
246
247
248
249
250
251
252 public static void setSocketTimeout(Configuration conf, int socketTimeout) {
253 conf.setInt(SOCKET_TIMEOUT, socketTimeout);
254 }
255
256
257
258
259 static int getSocketTimeout(Configuration conf) {
260 return conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT);
261 }
262
263
264 protected class Call {
265 final int id;
266 final Message param;
267
268
269
270
271 CellScanner cells;
272 Message response;
273
274 Message responseDefaultType;
275 IOException error;
276 boolean done;
277 long startTime;
278 final MethodDescriptor md;
279
280 protected Call(final MethodDescriptor md, Message param, final CellScanner cells,
281 final Message responseDefaultType) {
282 this.param = param;
283 this.md = md;
284 this.cells = cells;
285 this.startTime = System.currentTimeMillis();
286 this.responseDefaultType = responseDefaultType;
287 synchronized (RpcClient.this) {
288 this.id = counter++;
289 }
290 }
291
292 @Override
293 public String toString() {
294 return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
295 (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}";
296 }
297
298
299
300 protected synchronized void callComplete() {
301 this.done = true;
302 notify();
303 }
304
305
306
307
308
309
310 public void setException(IOException error) {
311 this.error = error;
312 callComplete();
313 }
314
315
316
317
318
319
320
321
322 public void setResponse(Message response, final CellScanner cells) {
323 this.response = response;
324 this.cells = cells;
325 callComplete();
326 }
327
328 public long getStartTime() {
329 return this.startTime;
330 }
331 }
332
333 protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
334 TokenSelector<? extends TokenIdentifier>> tokenHandlers =
335 new HashMap<AuthenticationProtos.TokenIdentifier.Kind, TokenSelector<? extends TokenIdentifier>>();
336 static {
337 tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
338 new AuthenticationTokenSelector());
339 }
340
341
342
343
344
345 protected Connection createConnection(ConnectionId remoteId, final Codec codec,
346 final CompressionCodec compressor)
347 throws IOException {
348 return new Connection(remoteId, codec, compressor);
349 }
350
351
352
353
354 protected class Connection extends Thread {
355 private ConnectionHeader header;
356 protected ConnectionId remoteId;
357 protected Socket socket = null;
358 protected DataInputStream in;
359 protected DataOutputStream out;
360 private InetSocketAddress server;
361 private String serverPrincipal;
362 private AuthMethod authMethod;
363 private boolean useSasl;
364 private Token<? extends TokenIdentifier> token;
365 private HBaseSaslRpcClient saslRpcClient;
366 private int reloginMaxBackoff;
367 private final Codec codec;
368 private final CompressionCodec compressor;
369
370
371 protected final ConcurrentSkipListMap<Integer, Call> calls =
372 new ConcurrentSkipListMap<Integer, Call>();
373 protected final AtomicLong lastActivity =
374 new AtomicLong();
375 protected final AtomicBoolean shouldCloseConnection =
376 new AtomicBoolean();
377 protected IOException closeException;
378
379 Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
380 throws IOException {
381 if (remoteId.getAddress().isUnresolved()) {
382 throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
383 }
384 this.server = remoteId.getAddress();
385 this.codec = codec;
386 this.compressor = compressor;
387
388 UserGroupInformation ticket = remoteId.getTicket().getUGI();
389 SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
390 this.useSasl = userProvider.isHBaseSecurityEnabled();
391 if (useSasl && securityInfo != null) {
392 AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
393 if (tokenKind != null) {
394 TokenSelector<? extends TokenIdentifier> tokenSelector =
395 tokenHandlers.get(tokenKind);
396 if (tokenSelector != null) {
397 token = tokenSelector.selectToken(new Text(clusterId),
398 ticket.getTokens());
399 } else if (LOG.isDebugEnabled()) {
400 LOG.debug("No token selector found for type "+tokenKind);
401 }
402 }
403 String serverKey = securityInfo.getServerPrincipal();
404 if (serverKey == null) {
405 throw new IOException(
406 "Can't obtain server Kerberos config key from SecurityInfo");
407 }
408 serverPrincipal = SecurityUtil.getServerPrincipal(
409 conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
410 if (LOG.isDebugEnabled()) {
411 LOG.debug("RPC Server Kerberos principal name for service="
412 + remoteId.getServiceName() + " is " + serverPrincipal);
413 }
414 }
415
416 if (!useSasl) {
417 authMethod = AuthMethod.SIMPLE;
418 } else if (token != null) {
419 authMethod = AuthMethod.DIGEST;
420 } else {
421 authMethod = AuthMethod.KERBEROS;
422 }
423
424 if (LOG.isDebugEnabled()) {
425 LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
426 ", sasl=" + useSasl);
427 }
428 reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
429 this.remoteId = remoteId;
430
431 ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
432 builder.setServiceName(remoteId.getServiceName());
433 UserInformation userInfoPB;
434 if ((userInfoPB = getUserInfo(ticket)) != null) {
435 builder.setUserInfo(userInfoPB);
436 }
437 if (this.codec != null) {
438 builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
439 }
440 if (this.compressor != null) {
441 builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
442 }
443 this.header = builder.build();
444
445 this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
446 remoteId.getAddress().toString() +
447 ((ticket==null)?" from an unknown user": (" from "
448 + ticket.getUserName())));
449 this.setDaemon(true);
450 }
451
452 private UserInformation getUserInfo(UserGroupInformation ugi) {
453 if (ugi == null || authMethod == AuthMethod.DIGEST) {
454
455 return null;
456 }
457 UserInformation.Builder userInfoPB = UserInformation.newBuilder();
458 if (authMethod == AuthMethod.KERBEROS) {
459
460 userInfoPB.setEffectiveUser(ugi.getUserName());
461 } else if (authMethod == AuthMethod.SIMPLE) {
462
463 userInfoPB.setEffectiveUser(ugi.getUserName());
464 if (ugi.getRealUser() != null) {
465 userInfoPB.setRealUser(ugi.getRealUser().getUserName());
466 }
467 }
468 return userInfoPB.build();
469 }
470
471
472 protected void touch() {
473 lastActivity.set(System.currentTimeMillis());
474 }
475
476
477
478
479
480
481
482
483
484 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
485 justification="Notify because new call available for processing")
486 protected synchronized void addCall(Call call) {
487
488
489
490 if (this.shouldCloseConnection.get()) {
491 if (this.closeException == null) {
492 call.setException(new IOException(
493 "Call " + call.id + " not added as the connection " + remoteId + " is closing"));
494 } else {
495 call.setException(this.closeException);
496 }
497 synchronized (call) {
498 call.notifyAll();
499 }
500 } else {
501 calls.put(call.id, call);
502 synchronized (call) {
503 notify();
504 }
505 }
506 }
507
508
509
510
511
512 protected class PingInputStream extends FilterInputStream {
513
514 protected PingInputStream(InputStream in) {
515 super(in);
516 }
517
518
519
520
521
522 private void handleTimeout(SocketTimeoutException e) throws IOException {
523 if (shouldCloseConnection.get() || !running.get() || remoteId.rpcTimeout > 0) {
524 throw e;
525 }
526 sendPing();
527 }
528
529
530
531
532
533
534 @Override
535 public int read() throws IOException {
536 do {
537 try {
538 return super.read();
539 } catch (SocketTimeoutException e) {
540 handleTimeout(e);
541 }
542 } while (true);
543 }
544
545
546
547
548
549
550
551 @Override
552 public int read(byte[] buf, int off, int len) throws IOException {
553 do {
554 try {
555 return super.read(buf, off, len);
556 } catch (SocketTimeoutException e) {
557 handleTimeout(e);
558 }
559 } while (true);
560 }
561 }
562
563 protected synchronized void setupConnection() throws IOException {
564 short ioFailures = 0;
565 short timeoutFailures = 0;
566 while (true) {
567 try {
568 this.socket = socketFactory.createSocket();
569 this.socket.setTcpNoDelay(tcpNoDelay);
570 this.socket.setKeepAlive(tcpKeepAlive);
571 if (localAddr != null) {
572 this.socket.bind(localAddr);
573 }
574
575 NetUtils.connect(this.socket, remoteId.getAddress(),
576 getSocketTimeout(conf));
577 if (remoteId.rpcTimeout > 0) {
578 pingInterval = remoteId.rpcTimeout;
579 }
580 this.socket.setSoTimeout(pingInterval);
581 return;
582 } catch (SocketTimeoutException toe) {
583
584
585
586 handleConnectionFailure(timeoutFailures++, maxRetries, toe);
587 } catch (IOException ie) {
588 handleConnectionFailure(ioFailures++, maxRetries, ie);
589 }
590 }
591 }
592
593 protected void closeConnection() {
594 if (socket == null) {
595 return;
596 }
597
598
599 try {
600 if (socket.getOutputStream() != null) {
601 socket.getOutputStream().close();
602 }
603 } catch (IOException ignored) {
604 }
605 try {
606 if (socket.getInputStream() != null) {
607 socket.getInputStream().close();
608 }
609 } catch (IOException ignored) {
610 }
611 try {
612 if (socket.getChannel() != null) {
613 socket.getChannel().close();
614 }
615 } catch (IOException ignored) {
616 }
617 try {
618 socket.close();
619 } catch (IOException e) {
620 LOG.warn("Not able to close a socket", e);
621 }
622
623
624
625 socket = null;
626 }
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643 private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
644 throws IOException {
645
646 closeConnection();
647
648
649 if (curRetries >= maxRetries) {
650 throw ioe;
651 }
652
653
654 try {
655 Thread.sleep(failureSleep);
656 } catch (InterruptedException ignored) {}
657
658 LOG.info("Retrying connect to server: " + remoteId.getAddress() +
659 " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
660 " time(s).");
661 }
662
663
664
665
666
667
668
669 protected synchronized boolean waitForWork() {
670 if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
671 long timeout = maxIdleTime - (System.currentTimeMillis()-lastActivity.get());
672 if (timeout>0) {
673 try {
674 wait(timeout);
675 } catch (InterruptedException ignored) {}
676 }
677 }
678
679 if (!calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
680 return true;
681 } else if (shouldCloseConnection.get()) {
682 return false;
683 } else if (calls.isEmpty()) {
684 markClosed(null);
685 return false;
686 } else {
687 markClosed((IOException)new IOException().initCause(
688 new InterruptedException()));
689 return false;
690 }
691 }
692
693 public InetSocketAddress getRemoteAddress() {
694 return remoteId.getAddress();
695 }
696
697
698
699
700 protected synchronized void sendPing() throws IOException {
701
702 long curTime = System.currentTimeMillis();
703 if ( curTime - lastActivity.get() >= pingInterval) {
704 lastActivity.set(curTime);
705
706 synchronized (this.out) {
707 out.writeInt(PING_CALL_ID);
708 out.flush();
709 }
710 }
711 }
712
713 @Override
714 public void run() {
715 if (LOG.isDebugEnabled()) {
716 LOG.debug(getName() + ": starting, connections " + connections.size());
717 }
718
719 try {
720 while (waitForWork()) {
721 readResponse();
722 }
723 } catch (Throwable t) {
724 LOG.warn(getName() + ": unexpected exception receiving call responses", t);
725 markClosed(new IOException("Unexpected exception receiving call responses", t));
726 }
727
728 close();
729
730 if (LOG.isDebugEnabled())
731 LOG.debug(getName() + ": stopped, connections " + connections.size());
732 }
733
734 private synchronized void disposeSasl() {
735 if (saslRpcClient != null) {
736 try {
737 saslRpcClient.dispose();
738 saslRpcClient = null;
739 } catch (IOException ioe) {
740 LOG.error("Error disposing of SASL client", ioe);
741 }
742 }
743 }
744
745 private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
746 UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
747 UserGroupInformation currentUser =
748 UserGroupInformation.getCurrentUser();
749 UserGroupInformation realUser = currentUser.getRealUser();
750 return authMethod == AuthMethod.KERBEROS &&
751 loginUser != null &&
752
753 loginUser.hasKerberosCredentials() &&
754
755
756 (loginUser.equals(currentUser) || loginUser.equals(realUser));
757 }
758
759 private synchronized boolean setupSaslConnection(final InputStream in2,
760 final OutputStream out2) throws IOException {
761 saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed);
762 return saslRpcClient.saslConnect(in2, out2);
763 }
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783 private synchronized void handleSaslConnectionFailure(
784 final int currRetries,
785 final int maxRetries, final Exception ex, final Random rand,
786 final UserGroupInformation user)
787 throws IOException, InterruptedException{
788 user.doAs(new PrivilegedExceptionAction<Object>() {
789 public Object run() throws IOException, InterruptedException {
790 closeConnection();
791 if (shouldAuthenticateOverKrb()) {
792 if (currRetries < maxRetries) {
793 LOG.debug("Exception encountered while connecting to " +
794 "the server : " + ex);
795
796 if (UserGroupInformation.isLoginKeytabBased()) {
797 UserGroupInformation.getLoginUser().reloginFromKeytab();
798 } else {
799 UserGroupInformation.getLoginUser().reloginFromTicketCache();
800 }
801 disposeSasl();
802
803
804
805
806 Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
807 return null;
808 } else {
809 String msg = "Couldn't setup connection for " +
810 UserGroupInformation.getLoginUser().getUserName() +
811 " to " + serverPrincipal;
812 LOG.warn(msg);
813 throw (IOException) new IOException(msg).initCause(ex);
814 }
815 } else {
816 LOG.warn("Exception encountered while connecting to " +
817 "the server : " + ex);
818 }
819 if (ex instanceof RemoteException) {
820 throw (RemoteException)ex;
821 }
822 if (ex instanceof SaslException) {
823 String msg = "SASL authentication failed." +
824 " The most likely cause is missing or invalid credentials." +
825 " Consider 'kinit'.";
826 LOG.fatal(msg, ex);
827 throw new RuntimeException(msg, ex);
828 }
829 throw new IOException(ex);
830 }
831 });
832 }
833
834 protected synchronized void setupIOstreams()
835 throws IOException, InterruptedException {
836 if (socket != null || shouldCloseConnection.get()) {
837 return;
838 }
839
840 if (failedServers.isFailedServer(remoteId.getAddress())) {
841 if (LOG.isDebugEnabled()) {
842 LOG.debug("Not trying to connect to " + server +
843 " this server is in the failed servers list");
844 }
845 IOException e = new FailedServerException(
846 "This server is in the failed servers list: " + server);
847 markClosed(e);
848 close();
849 throw e;
850 }
851
852 try {
853 if (LOG.isDebugEnabled()) {
854 LOG.debug("Connecting to " + server);
855 }
856 short numRetries = 0;
857 final short MAX_RETRIES = 5;
858 Random rand = null;
859 while (true) {
860 setupConnection();
861 InputStream inStream = NetUtils.getInputStream(socket);
862
863
864
865 OutputStream outStream = NetUtils.getOutputStream(socket, pingInterval);
866
867 writeConnectionHeaderPreamble(outStream);
868 if (useSasl) {
869 final InputStream in2 = inStream;
870 final OutputStream out2 = outStream;
871 UserGroupInformation ticket = remoteId.getTicket().getUGI();
872 if (authMethod == AuthMethod.KERBEROS) {
873 if (ticket != null && ticket.getRealUser() != null) {
874 ticket = ticket.getRealUser();
875 }
876 }
877 boolean continueSasl = false;
878 if (ticket == null) throw new FatalConnectionException("ticket/user is null");
879 try {
880 continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
881 @Override
882 public Boolean run() throws IOException {
883 return setupSaslConnection(in2, out2);
884 }
885 });
886 } catch (Exception ex) {
887 if (rand == null) {
888 rand = new Random();
889 }
890 handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
891 continue;
892 }
893 if (continueSasl) {
894
895 inStream = saslRpcClient.getInputStream(inStream);
896 outStream = saslRpcClient.getOutputStream(outStream);
897 } else {
898
899 authMethod = AuthMethod.SIMPLE;
900 useSasl = false;
901 }
902 }
903 this.in = new DataInputStream(new BufferedInputStream(new PingInputStream(inStream)));
904 this.out = new DataOutputStream(new BufferedOutputStream(outStream));
905
906 writeConnectionHeader();
907
908
909 touch();
910
911
912 start();
913 return;
914 }
915 } catch (Throwable t) {
916 failedServers.addToFailedServers(remoteId.address);
917 IOException e = null;
918 if (t instanceof IOException) {
919 e = (IOException)t;
920 markClosed(e);
921 } else {
922 e = new IOException("Could not set up IO Streams", t);
923 markClosed(e);
924 }
925 close();
926 throw e;
927 }
928 }
929
930
931
932
933 private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
934
935
936
937
938
939 int rpcHeaderLen = HConstants.RPC_HEADER.array().length;
940 byte [] preamble = new byte [rpcHeaderLen + 2];
941 System.arraycopy(HConstants.RPC_HEADER.array(), 0, preamble, 0, rpcHeaderLen);
942 preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
943 preamble[rpcHeaderLen + 1] = authMethod.code;
944 outStream.write(preamble);
945 outStream.flush();
946 }
947
948
949
950
951
952 private void writeConnectionHeader() throws IOException {
953 synchronized (this.out) {
954 this.out.writeInt(this.header.getSerializedSize());
955 this.header.writeTo(this.out);
956 this.out.flush();
957 }
958 }
959
960
961 protected synchronized void close() {
962 if (!shouldCloseConnection.get()) {
963 LOG.error(getName() + ": the connection is not in the closed state");
964 return;
965 }
966
967
968
969 synchronized (connections) {
970 connections.remove(remoteId, this);
971 }
972
973
974 if (this.out != null) {
975 synchronized(this.out) {
976 IOUtils.closeStream(out);
977 this.out = null;
978 }
979 }
980 IOUtils.closeStream(in);
981 this.in = null;
982 disposeSasl();
983
984
985 if (closeException == null) {
986 if (!calls.isEmpty()) {
987 LOG.warn(getName() + ": connection is closed for no cause and calls are not empty. " +
988 "#Calls: " + calls.size());
989
990
991 closeException = new IOException("Unexpected closed connection");
992 cleanupCalls();
993 }
994 } else {
995
996 if (LOG.isDebugEnabled()) {
997 LOG.debug(getName() + ": closing ipc connection to " + server + ": " +
998 closeException.getMessage(), closeException);
999 }
1000
1001
1002 cleanupCalls();
1003 }
1004 if (LOG.isDebugEnabled())
1005 LOG.debug(getName() + ": closed");
1006 }
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016 protected void writeRequest(Call call, final int priority) {
1017 if (shouldCloseConnection.get()) return;
1018 try {
1019 RequestHeader.Builder builder = RequestHeader.newBuilder();
1020 builder.setCallId(call.id);
1021 if (Trace.isTracing()) {
1022 Span s = Trace.currentSpan();
1023 builder.setTraceInfo(RPCTInfo.newBuilder().
1024 setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
1025 }
1026 builder.setMethodName(call.md.getName());
1027 builder.setRequestParam(call.param != null);
1028 ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells);
1029 if (cellBlock != null) {
1030 CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
1031 cellBlockBuilder.setLength(cellBlock.limit());
1032 builder.setCellBlockMeta(cellBlockBuilder.build());
1033 }
1034
1035 if (priority != 0) builder.setPriority(priority);
1036
1037 RequestHeader header = builder.build();
1038 synchronized (this.out) {
1039 IPCUtil.write(this.out, header, call.param, cellBlock);
1040 }
1041 if (LOG.isDebugEnabled()) {
1042 LOG.debug(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
1043 }
1044 } catch(IOException e) {
1045 markClosed(e);
1046 }
1047 }
1048
1049
1050
1051
1052 protected void readResponse() {
1053 if (shouldCloseConnection.get()) return;
1054 touch();
1055 int totalSize = -1;
1056 try {
1057
1058
1059 totalSize = in.readInt();
1060
1061
1062 ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
1063 int id = responseHeader.getCallId();
1064 if (LOG.isDebugEnabled()) {
1065 LOG.debug(getName() + ": got response header " +
1066 TextFormat.shortDebugString(responseHeader) + ", totalSize: " + totalSize + " bytes");
1067 }
1068 Call call = calls.get(id);
1069 if (call == null) {
1070
1071
1072
1073
1074
1075 int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader);
1076 int whatIsLeftToRead = totalSize - readSoFar;
1077 LOG.debug("Unknown callId: " + id + ", skipping over this response of " +
1078 whatIsLeftToRead + " bytes");
1079 IOUtils.skipFully(in, whatIsLeftToRead);
1080 }
1081 if (responseHeader.hasException()) {
1082 ExceptionResponse exceptionResponse = responseHeader.getException();
1083 RemoteException re = createRemoteException(exceptionResponse);
1084 if (isFatalConnectionException(exceptionResponse)) {
1085 markClosed(re);
1086 } else {
1087 if (call != null) call.setException(re);
1088 }
1089 } else {
1090 Message value = null;
1091
1092 if (call != null && call.responseDefaultType != null) {
1093 Builder builder = call.responseDefaultType.newBuilderForType();
1094 builder.mergeDelimitedFrom(in);
1095 value = builder.build();
1096 }
1097 CellScanner cellBlockScanner = null;
1098 if (responseHeader.hasCellBlockMeta()) {
1099 int size = responseHeader.getCellBlockMeta().getLength();
1100 byte [] cellBlock = new byte[size];
1101 IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
1102 cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock);
1103 }
1104
1105
1106 if (call != null) call.setResponse(value, cellBlockScanner);
1107 }
1108 if (call != null) calls.remove(id);
1109 } catch (IOException e) {
1110 if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
1111
1112
1113
1114 closeException = e;
1115 } else {
1116
1117 markClosed(e);
1118 }
1119 } finally {
1120 if (remoteId.rpcTimeout > 0) {
1121 cleanupCalls(remoteId.rpcTimeout);
1122 }
1123 }
1124 }
1125
1126
1127
1128
1129
1130 private boolean isFatalConnectionException(final ExceptionResponse e) {
1131 return e.getExceptionClassName().
1132 equals(FatalConnectionException.class.getName());
1133 }
1134
1135
1136
1137
1138
1139 private RemoteException createRemoteException(final ExceptionResponse e) {
1140 String innerExceptionClassName = e.getExceptionClassName();
1141 boolean doNotRetry = e.getDoNotRetry();
1142 return e.hasHostname()?
1143
1144 new RemoteWithExtrasException(innerExceptionClassName,
1145 e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry):
1146 new RemoteWithExtrasException(innerExceptionClassName,
1147 e.getStackTrace(), doNotRetry);
1148 }
1149
1150 protected synchronized void markClosed(IOException e) {
1151 if (shouldCloseConnection.compareAndSet(false, true)) {
1152 closeException = e;
1153 notifyAll();
1154 }
1155 }
1156
1157
1158 protected void cleanupCalls() {
1159 cleanupCalls(0);
1160 }
1161
1162 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
1163 justification="Notify because timedout")
1164 protected void cleanupCalls(long rpcTimeout) {
1165 Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
1166 while (itor.hasNext()) {
1167 Call c = itor.next().getValue();
1168 long waitTime = System.currentTimeMillis() - c.getStartTime();
1169 if (waitTime >= rpcTimeout) {
1170 if (this.closeException == null) {
1171
1172
1173
1174
1175
1176
1177 this.closeException = new CallTimeoutException("Call id=" + c.id +
1178 ", waitTime=" + waitTime + ", rpcTimetout=" + rpcTimeout);
1179 }
1180 c.setException(this.closeException);
1181 synchronized (c) {
1182 c.notifyAll();
1183 }
1184 itor.remove();
1185 } else {
1186 break;
1187 }
1188 }
1189 try {
1190 if (!calls.isEmpty()) {
1191 Call firstCall = calls.get(calls.firstKey());
1192 long maxWaitTime = System.currentTimeMillis() - firstCall.getStartTime();
1193 if (maxWaitTime < rpcTimeout) {
1194 rpcTimeout -= maxWaitTime;
1195 }
1196 }
1197 if (!shouldCloseConnection.get()) {
1198 closeException = null;
1199 setSocketTimeout(socket, (int) rpcTimeout);
1200 }
1201 } catch (SocketException e) {
1202 LOG.debug("Couldn't lower timeout, which may result in longer than expected calls");
1203 }
1204 }
1205 }
1206
1207 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
1208 justification="Presume sync not needed setting socket timeout")
1209 private static void setSocketTimeout(final Socket socket, final int rpcTimeout)
1210 throws java.net.SocketException {
1211 if (socket == null) return;
1212 socket.setSoTimeout(rpcTimeout);
1213 }
1214
1215
1216
1217
1218 @SuppressWarnings("serial")
1219 @InterfaceAudience.Public
1220 @InterfaceStability.Evolving
1221 public static class CallTimeoutException extends IOException {
1222 public CallTimeoutException(final String msg) {
1223 super(msg);
1224 }
1225 }
1226
1227
1228
1229
1230
1231
1232
1233 RpcClient(Configuration conf, String clusterId, SocketFactory factory) {
1234 this(conf, clusterId, factory, null);
1235 }
1236
1237
1238
1239
1240
1241
1242
1243
1244 RpcClient(Configuration conf, String clusterId, SocketFactory factory, SocketAddress localAddr) {
1245 this.maxIdleTime = conf.getInt("hbase.ipc.client.connection.maxidletime", 10000);
1246 this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
1247 this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
1248 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1249 this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
1250 this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
1251 this.pingInterval = getPingInterval(conf);
1252 this.ipcUtil = new IPCUtil(conf);
1253 this.conf = conf;
1254 this.codec = getCodec();
1255 this.compressor = getCompressor(conf);
1256 this.socketFactory = factory;
1257 this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
1258 this.connections = new PoolMap<ConnectionId, Connection>(getPoolType(conf), getPoolSize(conf));
1259 this.failedServers = new FailedServers(conf);
1260 this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
1261 IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
1262 this.localAddr = localAddr;
1263 this.userProvider = UserProvider.instantiate(conf);
1264 if (LOG.isDebugEnabled()) {
1265 LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor +
1266 ", tcpKeepAlive=" + this.tcpKeepAlive +
1267 ", tcpNoDelay=" + this.tcpNoDelay +
1268 ", maxIdleTime=" + this.maxIdleTime +
1269 ", maxRetries=" + this.maxRetries +
1270 ", fallbackAllowed=" + this.fallbackAllowed +
1271 ", ping interval=" + this.pingInterval + "ms" +
1272 ", bind address=" + (this.localAddr != null ? this.localAddr : "null"));
1273 }
1274 }
1275
1276
1277
1278
1279
1280
1281 public RpcClient(Configuration conf, String clusterId) {
1282 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null);
1283 }
1284
1285
1286
1287
1288
1289
1290
1291 public RpcClient(Configuration conf, String clusterId, SocketAddress localAddr) {
1292 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr);
1293 }
1294
1295
1296
1297
1298
1299 Codec getCodec() {
1300
1301
1302 String className = conf.get("hbase.client.rpc.codec", getDefaultCodec(this.conf));
1303 if (className == null || className.length() == 0) return null;
1304 try {
1305 return (Codec)Class.forName(className).newInstance();
1306 } catch (Exception e) {
1307 throw new RuntimeException("Failed getting codec " + className, e);
1308 }
1309 }
1310
1311 @VisibleForTesting
1312 public static String getDefaultCodec(final Configuration c) {
1313
1314
1315
1316 return c.get("hbase.client.default.rpc.codec", KeyValueCodec.class.getCanonicalName());
1317 }
1318
1319
1320
1321
1322
1323
1324 private static CompressionCodec getCompressor(final Configuration conf) {
1325 String className = conf.get("hbase.client.rpc.compressor", null);
1326 if (className == null || className.isEmpty()) return null;
1327 try {
1328 return (CompressionCodec)Class.forName(className).newInstance();
1329 } catch (Exception e) {
1330 throw new RuntimeException("Failed getting compressor " + className, e);
1331 }
1332 }
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349 protected static PoolType getPoolType(Configuration config) {
1350 return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE),
1351 PoolType.RoundRobin, PoolType.ThreadLocal);
1352 }
1353
1354
1355
1356
1357
1358
1359
1360
1361 protected static int getPoolSize(Configuration config) {
1362 return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1);
1363 }
1364
1365
1366
1367
1368
1369 SocketFactory getSocketFactory() {
1370 return socketFactory;
1371 }
1372
1373
1374
1375 public void stop() {
1376 if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client");
1377 if (!running.compareAndSet(true, false)) return;
1378
1379
1380 synchronized (connections) {
1381 for (Connection conn : connections.values()) {
1382 conn.interrupt();
1383 }
1384 }
1385
1386
1387 while (!connections.isEmpty()) {
1388 try {
1389 Thread.sleep(100);
1390 } catch (InterruptedException ignored) {
1391 }
1392 }
1393 }
1394
1395 Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1396 Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
1397 throws InterruptedException, IOException {
1398 return call(md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
1399 }
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419 Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1420 Message returnType, User ticket, InetSocketAddress addr,
1421 int rpcTimeout, int priority)
1422 throws InterruptedException, IOException {
1423 Call call = new Call(md, param, cells, returnType);
1424 Connection connection =
1425 getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
1426 connection.writeRequest(call, priority);
1427 boolean interrupted = false;
1428
1429 synchronized (call) {
1430 while (!call.done) {
1431 if (connection.shouldCloseConnection.get()) {
1432 throw new IOException("Unexpected closed connection");
1433 }
1434 try {
1435 call.wait(1000);
1436 } catch (InterruptedException ignored) {
1437
1438 interrupted = true;
1439 }
1440 }
1441
1442 if (interrupted) {
1443
1444 Thread.currentThread().interrupt();
1445 }
1446
1447 if (call.error != null) {
1448 if (call.error instanceof RemoteException) {
1449 call.error.fillInStackTrace();
1450 throw call.error;
1451 }
1452
1453 throw wrapException(addr, call.error);
1454 }
1455 return new Pair<Message, CellScanner>(call.response, call.cells);
1456 }
1457 }
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471 protected IOException wrapException(InetSocketAddress addr,
1472 IOException exception) {
1473 if (exception instanceof ConnectException) {
1474
1475 return (ConnectException)new ConnectException(
1476 "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
1477 } else if (exception instanceof SocketTimeoutException) {
1478 return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
1479 " failed because " + exception).initCause(exception);
1480 } else {
1481 return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
1482 exception).initCause(exception);
1483 }
1484 }
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494 public void cancelConnections(String hostname, int port, IOException ioe) {
1495 synchronized (connections) {
1496 for (Connection connection : connections.values()) {
1497 if (connection.isAlive() &&
1498 connection.getRemoteAddress().getPort() == port &&
1499 connection.getRemoteAddress().getHostName().equals(hostname)) {
1500 LOG.info("The server on " + hostname + ":" + port +
1501 " is dead - stopping the connection " + connection.remoteId);
1502 connection.closeConnection();
1503
1504
1505 }
1506 }
1507 }
1508 }
1509
1510
1511
1512 protected Connection getConnection(User ticket, Call call, InetSocketAddress addr,
1513 int rpcTimeout, final Codec codec, final CompressionCodec compressor)
1514 throws IOException, InterruptedException {
1515 if (!running.get()) throw new StoppedRpcClientException();
1516 Connection connection;
1517 ConnectionId remoteId =
1518 new ConnectionId(ticket, call.md.getService().getName(), addr, rpcTimeout);
1519 synchronized (connections) {
1520 connection = connections.get(remoteId);
1521 if (connection == null) {
1522 connection = createConnection(remoteId, this.codec, this.compressor);
1523 connections.put(remoteId, connection);
1524 }
1525 }
1526 connection.addCall(call);
1527
1528
1529
1530
1531
1532
1533
1534
1535 connection.setupIOstreams();
1536 return connection;
1537 }
1538
1539
1540
1541
1542
1543 protected static class ConnectionId {
1544 final InetSocketAddress address;
1545 final User ticket;
1546 final int rpcTimeout;
1547 private static final int PRIME = 16777619;
1548 final String serviceName;
1549
1550 ConnectionId(User ticket,
1551 String serviceName,
1552 InetSocketAddress address,
1553 int rpcTimeout) {
1554 this.address = address;
1555 this.ticket = ticket;
1556 this.rpcTimeout = rpcTimeout;
1557 this.serviceName = serviceName;
1558 }
1559
1560 String getServiceName() {
1561 return this.serviceName;
1562 }
1563
1564 InetSocketAddress getAddress() {
1565 return address;
1566 }
1567
1568 User getTicket() {
1569 return ticket;
1570 }
1571
1572 @Override
1573 public String toString() {
1574 return this.address.toString() + "/" + this.serviceName + "/" + this.ticket + "/" +
1575 this.rpcTimeout;
1576 }
1577
1578 @Override
1579 public boolean equals(Object obj) {
1580 if (obj instanceof ConnectionId) {
1581 ConnectionId id = (ConnectionId) obj;
1582 return address.equals(id.address) &&
1583 ((ticket != null && ticket.equals(id.ticket)) ||
1584 (ticket == id.ticket)) && rpcTimeout == id.rpcTimeout &&
1585 this.serviceName == id.serviceName;
1586 }
1587 return false;
1588 }
1589
1590 @Override
1591 public int hashCode() {
1592 int hashcode = (address.hashCode() +
1593 PRIME * (PRIME * this.serviceName.hashCode() ^
1594 (ticket == null ? 0 : ticket.hashCode()) )) ^
1595 rpcTimeout;
1596 return hashcode;
1597 }
1598 }
1599
1600 public static void setRpcTimeout(int t) {
1601 rpcTimeout.set(t);
1602 }
1603
1604 public static int getRpcTimeout() {
1605 return rpcTimeout.get();
1606 }
1607
1608
1609
1610
1611
1612 public static int getRpcTimeout(int defaultTimeout) {
1613 return Math.min(defaultTimeout, rpcTimeout.get());
1614 }
1615
1616 public static void resetRpcTimeout() {
1617 rpcTimeout.remove();
1618 }
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636 Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1637 Message param, Message returnType, final User ticket, final InetSocketAddress isa,
1638 final int rpcTimeout)
1639 throws ServiceException {
1640 long startTime = 0;
1641 if (LOG.isTraceEnabled()) {
1642 startTime = System.currentTimeMillis();
1643 }
1644 PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
1645 CellScanner cells = null;
1646 if (pcrc != null) {
1647 cells = pcrc.cellScanner();
1648
1649 pcrc.setCellScanner(null);
1650 }
1651 Pair<Message, CellScanner> val = null;
1652 try {
1653 val = call(md, param, cells, returnType, ticket, isa, rpcTimeout,
1654 pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS);
1655 if (pcrc != null) {
1656
1657 if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond());
1658 } else if (val.getSecond() != null) {
1659 throw new ServiceException("Client dropping data on the floor!");
1660 }
1661
1662 if (LOG.isTraceEnabled()) {
1663 long callTime = System.currentTimeMillis() - startTime;
1664 if (LOG.isTraceEnabled()) {
1665 LOG.trace("Call: " + md.getName() + ", callTime: " + callTime + "ms");
1666 }
1667 }
1668 return val.getFirst();
1669 } catch (Throwable e) {
1670 throw new ServiceException(e);
1671 }
1672 }
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682 public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn,
1683 final User ticket, final int rpcTimeout) {
1684 return new BlockingRpcChannelImplementation(this, sn, ticket, rpcTimeout);
1685 }
1686
1687
1688
1689
1690
1691 public static class BlockingRpcChannelImplementation implements BlockingRpcChannel {
1692 private final InetSocketAddress isa;
1693 private volatile RpcClient rpcClient;
1694 private final int rpcTimeout;
1695 private final User ticket;
1696
1697 protected BlockingRpcChannelImplementation(final RpcClient rpcClient, final ServerName sn,
1698 final User ticket, final int rpcTimeout) {
1699 this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
1700 this.rpcClient = rpcClient;
1701
1702
1703 this.rpcTimeout = getRpcTimeout(rpcTimeout);
1704 this.ticket = ticket;
1705 }
1706
1707 @Override
1708 public Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1709 Message param, Message returnType)
1710 throws ServiceException {
1711 return this.rpcClient.callBlockingMethod(md, controller, param, returnType, this.ticket,
1712 this.isa, this.rpcTimeout);
1713 }
1714 }
1715 }