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.DoNotRetryIOException;
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.Pair;
81 import org.apache.hadoop.hbase.util.PoolMap;
82 import org.apache.hadoop.hbase.util.PoolMap.PoolType;
83 import org.apache.hadoop.io.IOUtils;
84 import org.apache.hadoop.io.Text;
85 import org.apache.hadoop.io.compress.CompressionCodec;
86 import org.apache.hadoop.ipc.RemoteException;
87 import org.apache.hadoop.net.NetUtils;
88 import org.apache.hadoop.security.SecurityUtil;
89 import org.apache.hadoop.security.UserGroupInformation;
90 import org.apache.hadoop.security.token.Token;
91 import org.apache.hadoop.security.token.TokenIdentifier;
92 import org.apache.hadoop.security.token.TokenSelector;
93 import org.cloudera.htrace.Span;
94 import org.cloudera.htrace.Trace;
95
96 import com.google.common.annotations.VisibleForTesting;
97 import com.google.protobuf.BlockingRpcChannel;
98 import com.google.protobuf.Descriptors.MethodDescriptor;
99 import com.google.protobuf.Message;
100 import com.google.protobuf.Message.Builder;
101 import com.google.protobuf.RpcController;
102 import com.google.protobuf.ServiceException;
103 import com.google.protobuf.TextFormat;
104
105
106
107
108
109
110 @InterfaceAudience.Private
111 public class RpcClient {
112
113
114 public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RpcClient");
115 protected final PoolMap<ConnectionId, Connection> connections;
116
117 protected int counter;
118 protected final AtomicBoolean running = new AtomicBoolean(true);
119 final protected Configuration conf;
120 final protected int maxIdleTime;
121
122 final protected int maxRetries;
123 final protected long failureSleep;
124 protected final boolean tcpNoDelay;
125 protected final boolean tcpKeepAlive;
126 protected int pingInterval;
127 protected FailedServers failedServers;
128 private final Codec codec;
129 private final CompressionCodec compressor;
130 private final IPCUtil ipcUtil;
131
132 protected final SocketFactory socketFactory;
133 protected String clusterId;
134 protected final SocketAddress localAddr;
135
136 private final boolean fallbackAllowed;
137 private UserProvider userProvider;
138
139 final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
140 final private static String SOCKET_TIMEOUT = "ipc.socket.timeout";
141 final static int DEFAULT_PING_INTERVAL = 60000;
142 final static int DEFAULT_SOCKET_TIMEOUT = 20000;
143 final static int PING_CALL_ID = -1;
144
145 public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
146 public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
147
148 public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
149 "hbase.ipc.client.fallback-to-simple-auth-allowed";
150 public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
151
152
153
154
155
156
157 private static ThreadLocal<Integer> rpcTimeout = new ThreadLocal<Integer>() {
158 @Override
159 protected Integer initialValue() {
160 return HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
161 }
162 };
163
164
165
166
167 static class FailedServers {
168 private final LinkedList<Pair<Long, String>> failedServers = new
169 LinkedList<Pair<Long, java.lang.String>>();
170 private final int recheckServersTimeout;
171
172 FailedServers(Configuration conf) {
173 this.recheckServersTimeout = conf.getInt(
174 FAILED_SERVER_EXPIRY_KEY, FAILED_SERVER_EXPIRY_DEFAULT);
175 }
176
177
178
179
180 public synchronized void addToFailedServers(InetSocketAddress address) {
181 final long expiry = EnvironmentEdgeManager.currentTimeMillis() + recheckServersTimeout;
182 failedServers.addFirst(new Pair<Long, String>(expiry, address.toString()));
183 }
184
185
186
187
188
189
190 public synchronized boolean isFailedServer(final InetSocketAddress address) {
191 if (failedServers.isEmpty()) {
192 return false;
193 }
194
195 final String lookup = address.toString();
196 final long now = EnvironmentEdgeManager.currentTimeMillis();
197
198
199 Iterator<Pair<Long, String>> it = failedServers.iterator();
200 while (it.hasNext()) {
201 Pair<Long, String> cur = it.next();
202 if (cur.getFirst() < now) {
203 it.remove();
204 } else {
205 if (lookup.equals(cur.getSecond())) {
206 return true;
207 }
208 }
209 }
210
211 return false;
212 }
213 }
214
215 @SuppressWarnings("serial")
216 @InterfaceAudience.Public
217 @InterfaceStability.Evolving
218
219 public static class FailedServerException extends HBaseIOException {
220 public FailedServerException(String s) {
221 super(s);
222 }
223 }
224
225
226
227
228
229
230
231
232
233 public static void setPingInterval(Configuration conf, int pingInterval) {
234 conf.setInt(PING_INTERVAL_NAME, pingInterval);
235 }
236
237
238
239
240
241
242
243
244 static int getPingInterval(Configuration conf) {
245 return conf.getInt(PING_INTERVAL_NAME, DEFAULT_PING_INTERVAL);
246 }
247
248
249
250
251
252
253 public static void setSocketTimeout(Configuration conf, int socketTimeout) {
254 conf.setInt(SOCKET_TIMEOUT, socketTimeout);
255 }
256
257
258
259
260 static int getSocketTimeout(Configuration conf) {
261 return conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT);
262 }
263
264
265 protected class Call {
266 final int id;
267 final Message param;
268
269
270
271
272 CellScanner cells;
273 Message response;
274
275 Message responseDefaultType;
276 IOException error;
277 boolean done;
278 long startTime;
279 final MethodDescriptor md;
280
281 protected Call(final MethodDescriptor md, Message param, final CellScanner cells,
282 final Message responseDefaultType) {
283 this.param = param;
284 this.md = md;
285 this.cells = cells;
286 this.startTime = System.currentTimeMillis();
287 this.responseDefaultType = responseDefaultType;
288 synchronized (RpcClient.this) {
289 this.id = counter++;
290 }
291 }
292
293 @Override
294 public String toString() {
295 return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
296 (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}";
297 }
298
299
300
301 protected synchronized void callComplete() {
302 this.done = true;
303 notify();
304 }
305
306
307
308
309
310
311 public void setException(IOException error) {
312 this.error = error;
313 callComplete();
314 }
315
316
317
318
319
320
321
322
323 public void setResponse(Message response, final CellScanner cells) {
324 this.response = response;
325 this.cells = cells;
326 callComplete();
327 }
328
329 public long getStartTime() {
330 return this.startTime;
331 }
332 }
333
334 protected final static Map<AuthenticationProtos.TokenIdentifier.Kind,
335 TokenSelector<? extends TokenIdentifier>> tokenHandlers =
336 new HashMap<AuthenticationProtos.TokenIdentifier.Kind, TokenSelector<? extends TokenIdentifier>>();
337 static {
338 tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
339 new AuthenticationTokenSelector());
340 }
341
342
343
344
345
346 protected Connection createConnection(ConnectionId remoteId, final Codec codec,
347 final CompressionCodec compressor)
348 throws IOException {
349 return new Connection(remoteId, codec, compressor);
350 }
351
352
353
354
355 protected class Connection extends Thread {
356 private ConnectionHeader header;
357 protected ConnectionId remoteId;
358 protected Socket socket = null;
359 protected DataInputStream in;
360 protected DataOutputStream out;
361 private InetSocketAddress server;
362 private String serverPrincipal;
363 private AuthMethod authMethod;
364 private boolean useSasl;
365 private Token<? extends TokenIdentifier> token;
366 private HBaseSaslRpcClient saslRpcClient;
367 private int reloginMaxBackoff;
368 private final Codec codec;
369 private final CompressionCodec compressor;
370
371
372 protected final ConcurrentSkipListMap<Integer, Call> calls =
373 new ConcurrentSkipListMap<Integer, Call>();
374 protected final AtomicLong lastActivity =
375 new AtomicLong();
376 protected final AtomicBoolean shouldCloseConnection =
377 new AtomicBoolean();
378 protected IOException closeException;
379
380 Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
381 throws IOException {
382 if (remoteId.getAddress().isUnresolved()) {
383 throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
384 }
385 this.server = remoteId.getAddress();
386 this.codec = codec;
387 this.compressor = compressor;
388
389 UserGroupInformation ticket = remoteId.getTicket().getUGI();
390 SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
391 this.useSasl = userProvider.isHBaseSecurityEnabled();
392 if (useSasl && securityInfo != null) {
393 AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
394 if (tokenKind != null) {
395 TokenSelector<? extends TokenIdentifier> tokenSelector =
396 tokenHandlers.get(tokenKind);
397 if (tokenSelector != null) {
398 token = tokenSelector.selectToken(new Text(clusterId),
399 ticket.getTokens());
400 } else if (LOG.isDebugEnabled()) {
401 LOG.debug("No token selector found for type "+tokenKind);
402 }
403 }
404 String serverKey = securityInfo.getServerPrincipal();
405 if (serverKey == null) {
406 throw new IOException(
407 "Can't obtain server Kerberos config key from SecurityInfo");
408 }
409 serverPrincipal = SecurityUtil.getServerPrincipal(
410 conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
411 if (LOG.isDebugEnabled()) {
412 LOG.debug("RPC Server Kerberos principal name for service="
413 + remoteId.getServiceName() + " is " + serverPrincipal);
414 }
415 }
416
417 if (!useSasl) {
418 authMethod = AuthMethod.SIMPLE;
419 } else if (token != null) {
420 authMethod = AuthMethod.DIGEST;
421 } else {
422 authMethod = AuthMethod.KERBEROS;
423 }
424
425 if (LOG.isDebugEnabled()) {
426 LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
427 ", sasl=" + useSasl);
428 }
429 reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
430 this.remoteId = remoteId;
431
432 ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
433 builder.setServiceName(remoteId.getServiceName());
434 UserInformation userInfoPB;
435 if ((userInfoPB = getUserInfo(ticket)) != null) {
436 builder.setUserInfo(userInfoPB);
437 }
438 if (this.codec != null) {
439 builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
440 }
441 if (this.compressor != null) {
442 builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
443 }
444 this.header = builder.build();
445
446 this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
447 remoteId.getAddress().toString() +
448 ((ticket==null)?" from an unknown user": (" from "
449 + ticket.getUserName())));
450 this.setDaemon(true);
451 }
452
453 private UserInformation getUserInfo(UserGroupInformation ugi) {
454 if (ugi == null || authMethod == AuthMethod.DIGEST) {
455
456 return null;
457 }
458 UserInformation.Builder userInfoPB = UserInformation.newBuilder();
459 if (authMethod == AuthMethod.KERBEROS) {
460
461 userInfoPB.setEffectiveUser(ugi.getUserName());
462 } else if (authMethod == AuthMethod.SIMPLE) {
463
464 userInfoPB.setEffectiveUser(ugi.getUserName());
465 if (ugi.getRealUser() != null) {
466 userInfoPB.setRealUser(ugi.getRealUser().getUserName());
467 }
468 }
469 return userInfoPB.build();
470 }
471
472
473 protected void touch() {
474 lastActivity.set(System.currentTimeMillis());
475 }
476
477
478
479
480
481
482
483
484
485 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
486 justification="Notify because new call available for processing")
487 protected synchronized void addCall(Call call) {
488
489
490
491 if (this.shouldCloseConnection.get()) {
492 if (this.closeException == null) {
493 call.setException(new IOException(
494 "Call " + call.id + " not added as the connection " + remoteId + " is closing"));
495 } else {
496 call.setException(this.closeException);
497 }
498 synchronized (call) {
499 call.notifyAll();
500 }
501 } else {
502 calls.put(call.id, call);
503 synchronized (call) {
504 notify();
505 }
506 }
507 }
508
509
510
511
512
513 protected class PingInputStream extends FilterInputStream {
514
515 protected PingInputStream(InputStream in) {
516 super(in);
517 }
518
519
520
521
522
523 private void handleTimeout(SocketTimeoutException e) throws IOException {
524 if (shouldCloseConnection.get() || !running.get() || remoteId.rpcTimeout > 0) {
525 throw e;
526 }
527 sendPing();
528 }
529
530
531
532
533
534
535 @Override
536 public int read() throws IOException {
537 do {
538 try {
539 return super.read();
540 } catch (SocketTimeoutException e) {
541 handleTimeout(e);
542 }
543 } while (true);
544 }
545
546
547
548
549
550
551
552 @Override
553 public int read(byte[] buf, int off, int len) throws IOException {
554 do {
555 try {
556 return super.read(buf, off, len);
557 } catch (SocketTimeoutException e) {
558 handleTimeout(e);
559 }
560 } while (true);
561 }
562 }
563
564 protected synchronized void setupConnection() throws IOException {
565 short ioFailures = 0;
566 short timeoutFailures = 0;
567 while (true) {
568 try {
569 this.socket = socketFactory.createSocket();
570 this.socket.setTcpNoDelay(tcpNoDelay);
571 this.socket.setKeepAlive(tcpKeepAlive);
572 if (localAddr != null) {
573 this.socket.bind(localAddr);
574 }
575
576 NetUtils.connect(this.socket, remoteId.getAddress(),
577 getSocketTimeout(conf));
578 if (remoteId.rpcTimeout > 0) {
579 pingInterval = remoteId.rpcTimeout;
580 }
581 this.socket.setSoTimeout(pingInterval);
582 return;
583 } catch (SocketTimeoutException toe) {
584
585
586
587 handleConnectionFailure(timeoutFailures++, maxRetries, toe);
588 } catch (IOException ie) {
589 handleConnectionFailure(ioFailures++, maxRetries, ie);
590 }
591 }
592 }
593
594 protected void closeConnection() {
595 if (socket == null) {
596 return;
597 }
598
599
600 try {
601 if (socket.getOutputStream() != null) {
602 socket.getOutputStream().close();
603 }
604 } catch (IOException ignored) {
605 }
606 try {
607 if (socket.getInputStream() != null) {
608 socket.getInputStream().close();
609 }
610 } catch (IOException ignored) {
611 }
612 try {
613 if (socket.getChannel() != null) {
614 socket.getChannel().close();
615 }
616 } catch (IOException ignored) {
617 }
618 try {
619 socket.close();
620 } catch (IOException e) {
621 LOG.warn("Not able to close a socket", e);
622 }
623
624
625
626 socket = null;
627 }
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644 private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
645 throws IOException {
646
647 closeConnection();
648
649
650 if (curRetries >= maxRetries) {
651 throw ioe;
652 }
653
654
655 try {
656 Thread.sleep(failureSleep);
657 } catch (InterruptedException ignored) {}
658
659 LOG.info("Retrying connect to server: " + remoteId.getAddress() +
660 " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
661 " time(s).");
662 }
663
664
665
666
667
668
669
670 protected synchronized boolean waitForWork() {
671 if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
672 long timeout = maxIdleTime - (System.currentTimeMillis()-lastActivity.get());
673 if (timeout>0) {
674 try {
675 wait(timeout);
676 } catch (InterruptedException ignored) {}
677 }
678 }
679
680 if (!calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
681 return true;
682 } else if (shouldCloseConnection.get()) {
683 return false;
684 } else if (calls.isEmpty()) {
685 markClosed(null);
686 return false;
687 } else {
688 markClosed((IOException)new IOException().initCause(
689 new InterruptedException()));
690 return false;
691 }
692 }
693
694 public InetSocketAddress getRemoteAddress() {
695 return remoteId.getAddress();
696 }
697
698
699
700
701 protected synchronized void sendPing() throws IOException {
702
703 long curTime = System.currentTimeMillis();
704 if ( curTime - lastActivity.get() >= pingInterval) {
705 lastActivity.set(curTime);
706
707 synchronized (this.out) {
708 out.writeInt(PING_CALL_ID);
709 out.flush();
710 }
711 }
712 }
713
714 @Override
715 public void run() {
716 if (LOG.isDebugEnabled()) {
717 LOG.debug(getName() + ": starting, connections " + connections.size());
718 }
719
720 try {
721 while (waitForWork()) {
722 readResponse();
723 }
724 } catch (Throwable t) {
725 LOG.warn(getName() + ": unexpected exception receiving call responses", t);
726 markClosed(new IOException("Unexpected exception receiving call responses", t));
727 }
728
729 close();
730
731 if (LOG.isDebugEnabled())
732 LOG.debug(getName() + ": stopped, connections " + connections.size());
733 }
734
735 private synchronized void disposeSasl() {
736 if (saslRpcClient != null) {
737 try {
738 saslRpcClient.dispose();
739 saslRpcClient = null;
740 } catch (IOException ioe) {
741 LOG.error("Error disposing of SASL client", ioe);
742 }
743 }
744 }
745
746 private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
747 UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
748 UserGroupInformation currentUser =
749 UserGroupInformation.getCurrentUser();
750 UserGroupInformation realUser = currentUser.getRealUser();
751 return authMethod == AuthMethod.KERBEROS &&
752 loginUser != null &&
753
754 loginUser.hasKerberosCredentials() &&
755
756
757 (loginUser.equals(currentUser) || loginUser.equals(realUser));
758 }
759
760 private synchronized boolean setupSaslConnection(final InputStream in2,
761 final OutputStream out2) throws IOException {
762 saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed);
763 return saslRpcClient.saslConnect(in2, out2);
764 }
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784 private synchronized void handleSaslConnectionFailure(
785 final int currRetries,
786 final int maxRetries, final Exception ex, final Random rand,
787 final UserGroupInformation user)
788 throws IOException, InterruptedException{
789 user.doAs(new PrivilegedExceptionAction<Object>() {
790 public Object run() throws IOException, InterruptedException {
791 closeConnection();
792 if (shouldAuthenticateOverKrb()) {
793 if (currRetries < maxRetries) {
794 LOG.debug("Exception encountered while connecting to " +
795 "the server : " + ex);
796
797 if (UserGroupInformation.isLoginKeytabBased()) {
798 UserGroupInformation.getLoginUser().reloginFromKeytab();
799 } else {
800 UserGroupInformation.getLoginUser().reloginFromTicketCache();
801 }
802 disposeSasl();
803
804
805
806
807 Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
808 return null;
809 } else {
810 String msg = "Couldn't setup connection for " +
811 UserGroupInformation.getLoginUser().getUserName() +
812 " to " + serverPrincipal;
813 LOG.warn(msg);
814 throw (IOException) new IOException(msg).initCause(ex);
815 }
816 } else {
817 LOG.warn("Exception encountered while connecting to " +
818 "the server : " + ex);
819 }
820 if (ex instanceof RemoteException) {
821 throw (RemoteException)ex;
822 }
823 if (ex instanceof SaslException) {
824 String msg = "SASL authentication failed." +
825 " The most likely cause is missing or invalid credentials." +
826 " Consider 'kinit'.";
827 LOG.fatal(msg, ex);
828 throw new RuntimeException(msg, ex);
829 }
830 throw new IOException(ex);
831 }
832 });
833 }
834
835 protected synchronized void setupIOstreams()
836 throws IOException, InterruptedException {
837 if (socket != null || shouldCloseConnection.get()) {
838 return;
839 }
840
841 if (failedServers.isFailedServer(remoteId.getAddress())) {
842 if (LOG.isDebugEnabled()) {
843 LOG.debug("Not trying to connect to " + server +
844 " this server is in the failed servers list");
845 }
846 IOException e = new FailedServerException(
847 "This server is in the failed servers list: " + server);
848 markClosed(e);
849 close();
850 throw e;
851 }
852
853 try {
854 if (LOG.isDebugEnabled()) {
855 LOG.debug("Connecting to " + server);
856 }
857 short numRetries = 0;
858 final short MAX_RETRIES = 5;
859 Random rand = null;
860 while (true) {
861 setupConnection();
862 InputStream inStream = NetUtils.getInputStream(socket);
863
864
865
866 OutputStream outStream = NetUtils.getOutputStream(socket, pingInterval);
867
868 writeConnectionHeaderPreamble(outStream);
869 if (useSasl) {
870 final InputStream in2 = inStream;
871 final OutputStream out2 = outStream;
872 UserGroupInformation ticket = remoteId.getTicket().getUGI();
873 if (authMethod == AuthMethod.KERBEROS) {
874 if (ticket != null && ticket.getRealUser() != null) {
875 ticket = ticket.getRealUser();
876 }
877 }
878 boolean continueSasl = false;
879 if (ticket == null) throw new FatalConnectionException("ticket/user is null");
880 try {
881 continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
882 @Override
883 public Boolean run() throws IOException {
884 return setupSaslConnection(in2, out2);
885 }
886 });
887 } catch (Exception ex) {
888 if (rand == null) {
889 rand = new Random();
890 }
891 handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
892 continue;
893 }
894 if (continueSasl) {
895
896 inStream = saslRpcClient.getInputStream(inStream);
897 outStream = saslRpcClient.getOutputStream(outStream);
898 } else {
899
900 authMethod = AuthMethod.SIMPLE;
901 useSasl = false;
902 }
903 }
904 this.in = new DataInputStream(new BufferedInputStream(new PingInputStream(inStream)));
905 this.out = new DataOutputStream(new BufferedOutputStream(outStream));
906
907 writeConnectionHeader();
908
909
910 touch();
911
912
913 start();
914 return;
915 }
916 } catch (Throwable t) {
917 failedServers.addToFailedServers(remoteId.address);
918 IOException e = null;
919 if (t instanceof LinkageError) {
920
921 e = new DoNotRetryIOException(t);
922 markClosed(e);
923 } else 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.remove(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 if (LOG.isDebugEnabled()) {
1270 LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor +
1271 ", tcpKeepAlive=" + this.tcpKeepAlive +
1272 ", tcpNoDelay=" + this.tcpNoDelay +
1273 ", maxIdleTime=" + this.maxIdleTime +
1274 ", maxRetries=" + this.maxRetries +
1275 ", fallbackAllowed=" + this.fallbackAllowed +
1276 ", ping interval=" + this.pingInterval + "ms" +
1277 ", bind address=" + (this.localAddr != null ? this.localAddr : "null"));
1278 }
1279 }
1280
1281
1282
1283
1284
1285
1286 public RpcClient(Configuration conf, String clusterId) {
1287 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null);
1288 }
1289
1290
1291
1292
1293
1294
1295
1296 public RpcClient(Configuration conf, String clusterId, SocketAddress localAddr) {
1297 this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr);
1298 }
1299
1300
1301
1302
1303
1304 Codec getCodec() {
1305
1306
1307 String className = conf.get("hbase.client.rpc.codec", getDefaultCodec(this.conf));
1308 if (className == null || className.length() == 0) return null;
1309 try {
1310 return (Codec)Class.forName(className).newInstance();
1311 } catch (Exception e) {
1312 throw new RuntimeException("Failed getting codec " + className, e);
1313 }
1314 }
1315
1316 @VisibleForTesting
1317 public static String getDefaultCodec(final Configuration c) {
1318
1319
1320
1321 return c.get("hbase.client.default.rpc.codec", KeyValueCodec.class.getCanonicalName());
1322 }
1323
1324
1325
1326
1327
1328
1329 private static CompressionCodec getCompressor(final Configuration conf) {
1330 String className = conf.get("hbase.client.rpc.compressor", null);
1331 if (className == null || className.isEmpty()) return null;
1332 try {
1333 return (CompressionCodec)Class.forName(className).newInstance();
1334 } catch (Exception e) {
1335 throw new RuntimeException("Failed getting compressor " + className, e);
1336 }
1337 }
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354 protected static PoolType getPoolType(Configuration config) {
1355 return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE),
1356 PoolType.RoundRobin, PoolType.ThreadLocal);
1357 }
1358
1359
1360
1361
1362
1363
1364
1365
1366 protected static int getPoolSize(Configuration config) {
1367 return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1);
1368 }
1369
1370
1371
1372
1373
1374 SocketFactory getSocketFactory() {
1375 return socketFactory;
1376 }
1377
1378
1379
1380 public void stop() {
1381 if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client");
1382 if (!running.compareAndSet(true, false)) return;
1383
1384
1385 synchronized (connections) {
1386 for (Connection conn : connections.values()) {
1387 conn.interrupt();
1388 }
1389 }
1390
1391
1392 while (!connections.isEmpty()) {
1393 try {
1394 Thread.sleep(100);
1395 } catch (InterruptedException ignored) {
1396 }
1397 }
1398 }
1399
1400 Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1401 Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
1402 throws InterruptedException, IOException {
1403 return call(md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
1404 }
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424 Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
1425 Message returnType, User ticket, InetSocketAddress addr,
1426 int rpcTimeout, int priority)
1427 throws InterruptedException, IOException {
1428 Call call = new Call(md, param, cells, returnType);
1429 Connection connection =
1430 getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
1431 connection.writeRequest(call, priority);
1432 boolean interrupted = false;
1433
1434 synchronized (call) {
1435 while (!call.done) {
1436 if (connection.shouldCloseConnection.get()) {
1437 throw new IOException("Unexpected closed connection");
1438 }
1439 try {
1440 call.wait(1000);
1441 } catch (InterruptedException ignored) {
1442
1443 interrupted = true;
1444 }
1445 }
1446
1447 if (interrupted) {
1448
1449 Thread.currentThread().interrupt();
1450 }
1451
1452 if (call.error != null) {
1453 if (call.error instanceof RemoteException) {
1454 call.error.fillInStackTrace();
1455 throw call.error;
1456 }
1457
1458 throw wrapException(addr, call.error);
1459 }
1460 return new Pair<Message, CellScanner>(call.response, call.cells);
1461 }
1462 }
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476 protected IOException wrapException(InetSocketAddress addr,
1477 IOException exception) {
1478 if (exception instanceof ConnectException) {
1479
1480 return (ConnectException)new ConnectException(
1481 "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
1482 } else if (exception instanceof SocketTimeoutException) {
1483 return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
1484 " failed because " + exception).initCause(exception);
1485 } else {
1486 return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
1487 exception).initCause(exception);
1488 }
1489 }
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499 public void cancelConnections(String hostname, int port, IOException ioe) {
1500 synchronized (connections) {
1501 for (Connection connection : connections.values()) {
1502 if (connection.isAlive() &&
1503 connection.getRemoteAddress().getPort() == port &&
1504 connection.getRemoteAddress().getHostName().equals(hostname)) {
1505 LOG.info("The server on " + hostname + ":" + port +
1506 " is dead - stopping the connection " + connection.remoteId);
1507 connection.closeConnection();
1508
1509
1510 }
1511 }
1512 }
1513 }
1514
1515
1516
1517 protected Connection getConnection(User ticket, Call call, InetSocketAddress addr,
1518 int rpcTimeout, final Codec codec, final CompressionCodec compressor)
1519 throws IOException, InterruptedException {
1520 if (!running.get()) throw new StoppedRpcClientException();
1521 Connection connection;
1522 ConnectionId remoteId =
1523 new ConnectionId(ticket, call.md.getService().getName(), addr, rpcTimeout);
1524 synchronized (connections) {
1525 connection = connections.get(remoteId);
1526 if (connection == null) {
1527 connection = createConnection(remoteId, this.codec, this.compressor);
1528 connections.put(remoteId, connection);
1529 }
1530 }
1531 connection.addCall(call);
1532
1533
1534
1535
1536
1537
1538
1539
1540 connection.setupIOstreams();
1541 return connection;
1542 }
1543
1544
1545
1546
1547
1548 protected static class ConnectionId {
1549 final InetSocketAddress address;
1550 final User ticket;
1551 final int rpcTimeout;
1552 private static final int PRIME = 16777619;
1553 final String serviceName;
1554
1555 ConnectionId(User ticket,
1556 String serviceName,
1557 InetSocketAddress address,
1558 int rpcTimeout) {
1559 this.address = address;
1560 this.ticket = ticket;
1561 this.rpcTimeout = rpcTimeout;
1562 this.serviceName = serviceName;
1563 }
1564
1565 String getServiceName() {
1566 return this.serviceName;
1567 }
1568
1569 InetSocketAddress getAddress() {
1570 return address;
1571 }
1572
1573 User getTicket() {
1574 return ticket;
1575 }
1576
1577 @Override
1578 public String toString() {
1579 return this.address.toString() + "/" + this.serviceName + "/" + this.ticket + "/" +
1580 this.rpcTimeout;
1581 }
1582
1583 @Override
1584 public boolean equals(Object obj) {
1585 if (obj instanceof ConnectionId) {
1586 ConnectionId id = (ConnectionId) obj;
1587 return address.equals(id.address) &&
1588 ((ticket != null && ticket.equals(id.ticket)) ||
1589 (ticket == id.ticket)) && rpcTimeout == id.rpcTimeout &&
1590 this.serviceName == id.serviceName;
1591 }
1592 return false;
1593 }
1594
1595 @Override
1596 public int hashCode() {
1597 int hashcode = (address.hashCode() +
1598 PRIME * (PRIME * this.serviceName.hashCode() ^
1599 (ticket == null ? 0 : ticket.hashCode()) )) ^
1600 rpcTimeout;
1601 return hashcode;
1602 }
1603 }
1604
1605 public static void setRpcTimeout(int t) {
1606 rpcTimeout.set(t);
1607 }
1608
1609 public static int getRpcTimeout() {
1610 return rpcTimeout.get();
1611 }
1612
1613
1614
1615
1616
1617 public static int getRpcTimeout(int defaultTimeout) {
1618 return Math.min(defaultTimeout, rpcTimeout.get());
1619 }
1620
1621 public static void resetRpcTimeout() {
1622 rpcTimeout.remove();
1623 }
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641 Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1642 Message param, Message returnType, final User ticket, final InetSocketAddress isa,
1643 final int rpcTimeout)
1644 throws ServiceException {
1645 long startTime = 0;
1646 if (LOG.isTraceEnabled()) {
1647 startTime = System.currentTimeMillis();
1648 }
1649 PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
1650 CellScanner cells = null;
1651 if (pcrc != null) {
1652 cells = pcrc.cellScanner();
1653
1654 pcrc.setCellScanner(null);
1655 }
1656 Pair<Message, CellScanner> val = null;
1657 try {
1658 val = call(md, param, cells, returnType, ticket, isa, rpcTimeout,
1659 pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS);
1660 if (pcrc != null) {
1661
1662 if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond());
1663 } else if (val.getSecond() != null) {
1664 throw new ServiceException("Client dropping data on the floor!");
1665 }
1666
1667 if (LOG.isTraceEnabled()) {
1668 long callTime = System.currentTimeMillis() - startTime;
1669 if (LOG.isTraceEnabled()) {
1670 LOG.trace("Call: " + md.getName() + ", callTime: " + callTime + "ms");
1671 }
1672 }
1673 return val.getFirst();
1674 } catch (Throwable e) {
1675 throw new ServiceException(e);
1676 }
1677 }
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687 public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn,
1688 final User ticket, final int rpcTimeout) {
1689 return new BlockingRpcChannelImplementation(this, sn, ticket, rpcTimeout);
1690 }
1691
1692
1693
1694
1695
1696 public static class BlockingRpcChannelImplementation implements BlockingRpcChannel {
1697 private final InetSocketAddress isa;
1698 private volatile RpcClient rpcClient;
1699 private final int rpcTimeout;
1700 private final User ticket;
1701
1702 protected BlockingRpcChannelImplementation(final RpcClient rpcClient, final ServerName sn,
1703 final User ticket, final int rpcTimeout) {
1704 this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort());
1705 this.rpcClient = rpcClient;
1706
1707
1708 this.rpcTimeout = getRpcTimeout(rpcTimeout);
1709 this.ticket = ticket;
1710 }
1711
1712 @Override
1713 public Message callBlockingMethod(MethodDescriptor md, RpcController controller,
1714 Message param, Message returnType)
1715 throws ServiceException {
1716 return this.rpcClient.callBlockingMethod(md, controller, param, returnType, this.ticket,
1717 this.isa, this.rpcTimeout);
1718 }
1719 }
1720 }