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