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