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