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