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