1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.ipc;
22
23 import java.io.BufferedInputStream;
24 import java.io.BufferedOutputStream;
25 import java.io.DataInputStream;
26 import java.io.DataOutputStream;
27 import java.io.FilterInputStream;
28 import java.io.IOException;
29 import java.io.InputStream;
30 import java.net.ConnectException;
31 import java.net.InetSocketAddress;
32 import java.net.Socket;
33 import java.net.SocketException;
34 import java.net.SocketTimeoutException;
35 import java.net.UnknownHostException;
36 import java.util.Iterator;
37 import java.util.LinkedList;
38 import java.util.Map.Entry;
39 import java.util.concurrent.ConcurrentSkipListMap;
40 import java.util.concurrent.atomic.AtomicBoolean;
41 import java.util.concurrent.atomic.AtomicLong;
42
43 import javax.net.SocketFactory;
44
45 import org.apache.commons.logging.Log;
46 import org.apache.commons.logging.LogFactory;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.hbase.HConstants;
49 import org.apache.hadoop.hbase.security.User;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
52 import org.apache.hadoop.hbase.util.Pair;
53 import org.apache.hadoop.hbase.util.PoolMap;
54 import org.apache.hadoop.hbase.util.PoolMap.PoolType;
55 import org.apache.hadoop.io.DataOutputBuffer;
56 import org.apache.hadoop.io.IOUtils;
57 import org.apache.hadoop.io.Writable;
58 import org.apache.hadoop.io.WritableUtils;
59 import org.apache.hadoop.ipc.RemoteException;
60 import org.apache.hadoop.net.NetUtils;
61 import org.apache.hadoop.util.ReflectionUtils;
62
63
64
65
66
67
68
69
70
71
72 public class HBaseClient {
73
74 private static final Log LOG = LogFactory
75 .getLog("org.apache.hadoop.ipc.HBaseClient");
76 protected final PoolMap<ConnectionId, Connection> connections;
77
78 protected final Class<? extends Writable> valueClass;
79 protected int counter;
80 protected final AtomicBoolean running = new AtomicBoolean(true);
81 final protected Configuration conf;
82 final protected int maxIdleTime;
83
84 final protected int maxRetries;
85 final protected long failureSleep;
86 protected final boolean tcpNoDelay;
87 protected final boolean tcpKeepAlive;
88 protected int pingInterval;
89 protected int socketTimeout;
90 protected final InetSocketAddress bindAddress;
91 protected FailedServers failedServers;
92
93 protected final SocketFactory socketFactory;
94 protected String clusterId;
95
96 final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
97 final private static String SOCKET_TIMEOUT = "ipc.socket.timeout";
98 final static int DEFAULT_PING_INTERVAL = 60000;
99 final static int DEFAULT_SOCKET_TIMEOUT = 20000;
100 final static int PING_CALL_ID = -1;
101
102 public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
103 public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
104
105
106
107
108 static class FailedServers {
109 private final LinkedList<Pair<Long, String>> failedServers = new
110 LinkedList<Pair<Long, String>>();
111 private final int recheckServersTimeout;
112
113 FailedServers(Configuration conf) {
114 this.recheckServersTimeout = conf.getInt(
115 FAILED_SERVER_EXPIRY_KEY, FAILED_SERVER_EXPIRY_DEFAULT);
116 }
117
118
119
120
121 public synchronized void addToFailedServers(InetSocketAddress address) {
122 final long expiry = EnvironmentEdgeManager.currentTimeMillis() + recheckServersTimeout;
123 failedServers.addFirst(new Pair<Long, String>(expiry, address.toString()));
124 }
125
126
127
128
129
130
131 public synchronized boolean isFailedServer(final InetSocketAddress address) {
132 if (failedServers.isEmpty()) {
133 return false;
134 }
135
136 final String lookup = address.toString();
137 final long now = EnvironmentEdgeManager.currentTimeMillis();
138
139
140 Iterator<Pair<Long, String>> it = failedServers.iterator();
141 while (it.hasNext()) {
142 Pair<Long, String> cur = it.next();
143 if (cur.getFirst() < now) {
144 it.remove();
145 } else {
146 if (lookup.equals(cur.getSecond())) {
147 return true;
148 }
149 }
150 }
151
152 return false;
153 }
154
155 }
156
157 public static class FailedServerException extends IOException {
158 public FailedServerException(String s) {
159 super(s);
160 }
161 }
162
163
164
165
166
167
168
169
170 public static void setPingInterval(Configuration conf, int pingInterval) {
171 conf.setInt(PING_INTERVAL_NAME, pingInterval);
172 }
173
174
175
176
177
178
179
180
181 static int getPingInterval(Configuration conf) {
182 return conf.getInt(PING_INTERVAL_NAME, DEFAULT_PING_INTERVAL);
183 }
184
185
186
187
188
189
190 public static void setSocketTimeout(Configuration conf, int socketTimeout) {
191 conf.setInt(SOCKET_TIMEOUT, socketTimeout);
192 }
193
194
195
196
197 static int getSocketTimeout(Configuration conf) {
198 return conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT);
199 }
200
201
202 protected class Call {
203 final int id;
204 final Writable param;
205 Writable value;
206 IOException error;
207 boolean done;
208 long startTime;
209
210 protected Call(Writable param) {
211 this.param = param;
212 this.startTime = System.currentTimeMillis();
213 synchronized (HBaseClient.this) {
214 this.id = counter++;
215 }
216 }
217
218
219
220 protected synchronized void callComplete() {
221 this.done = true;
222 notify();
223 }
224
225
226
227
228
229
230 public synchronized void setException(IOException error) {
231 this.error = error;
232 callComplete();
233 }
234
235
236
237
238
239
240 public synchronized void setValue(Writable value) {
241 this.value = value;
242 callComplete();
243 }
244
245 public long getStartTime() {
246 return this.startTime;
247 }
248 }
249
250
251
252
253
254
255 protected Connection createConnection(ConnectionId remoteId) throws IOException {
256 return new Connection(remoteId);
257 }
258
259
260
261
262 protected class Connection extends Thread {
263 private ConnectionHeader header;
264 protected ConnectionId remoteId;
265 protected Socket socket = null;
266 protected DataInputStream in;
267 protected DataOutputStream out;
268
269
270 protected final ConcurrentSkipListMap<Integer, Call> calls = new ConcurrentSkipListMap<Integer, Call>();
271 protected final AtomicLong lastActivity = new AtomicLong();
272 protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();
273 protected IOException closeException;
274
275 Connection(ConnectionId remoteId) throws IOException {
276 if (remoteId.getAddress().isUnresolved()) {
277 throw new UnknownHostException("unknown host: " +
278 remoteId.getAddress().getHostName());
279 }
280 this.remoteId = remoteId;
281 User ticket = remoteId.getTicket();
282 Class<? extends VersionedProtocol> protocol = remoteId.getProtocol();
283
284 header = new ConnectionHeader(
285 protocol == null ? null : protocol.getName(), ticket);
286
287 this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
288 remoteId.getAddress().toString() +
289 ((ticket==null)?" from an unknown user": (" from " + ticket.getName())));
290 this.setDaemon(true);
291 }
292
293
294 protected void touch() {
295 lastActivity.set(System.currentTimeMillis());
296 }
297
298
299
300
301
302
303
304
305
306 protected synchronized void addCall(Call call) {
307
308
309
310 if (this.shouldCloseConnection.get()) {
311 if (this.closeException == null) {
312 call.setException(new IOException(
313 "Call " + call.id + " not added as the connection " + remoteId + " is closing"));
314 } else {
315 call.setException(this.closeException);
316 }
317 synchronized (call) {
318 call.notifyAll();
319 }
320 } else {
321 calls.put(call.id, call);
322 notify();
323 }
324 }
325
326
327
328
329
330 protected class PingInputStream extends FilterInputStream {
331
332 protected PingInputStream(InputStream in) {
333 super(in);
334 }
335
336
337
338
339
340 private void handleTimeout(SocketTimeoutException e) throws IOException {
341 if (shouldCloseConnection.get() || !running.get() ||
342 remoteId.rpcTimeout > 0) {
343 throw e;
344 }
345 sendPing();
346 }
347
348
349
350
351
352
353 @Override
354 public int read() throws IOException {
355 do {
356 try {
357 return super.read();
358 } catch (SocketTimeoutException e) {
359 handleTimeout(e);
360 }
361 } while (true);
362 }
363
364
365
366
367
368
369
370 @Override
371 public int read(byte[] buf, int off, int len) throws IOException {
372 do {
373 try {
374 return super.read(buf, off, len);
375 } catch (SocketTimeoutException e) {
376 handleTimeout(e);
377 }
378 } while (true);
379 }
380 }
381
382 protected synchronized void setupConnection() throws IOException {
383 short ioFailures = 0;
384 short timeoutFailures = 0;
385 while (true) {
386 try {
387 this.socket = socketFactory.createSocket();
388 this.socket.setTcpNoDelay(tcpNoDelay);
389 this.socket.setKeepAlive(tcpKeepAlive);
390 if (bindAddress != null) this.socket.bind(bindAddress);
391
392 NetUtils.connect(this.socket, remoteId.getAddress(),
393 getSocketTimeout(conf));
394 if (remoteId.rpcTimeout > 0) {
395 pingInterval = remoteId.rpcTimeout;
396 }
397 this.socket.setSoTimeout(pingInterval);
398 return;
399 } catch (SocketTimeoutException toe) {
400
401
402
403 handleConnectionFailure(timeoutFailures++, maxRetries, toe);
404 } catch (IOException ie) {
405 handleConnectionFailure(ioFailures++, maxRetries, ie);
406 }
407 }
408 }
409
410
411
412
413
414
415 protected synchronized void setupIOstreams()
416 throws IOException, InterruptedException {
417
418 if (socket != null || shouldCloseConnection.get()) {
419 return;
420 }
421
422 if (failedServers.isFailedServer(remoteId.getAddress())) {
423 if (LOG.isDebugEnabled()) {
424 LOG.debug("Not trying to connect to " + remoteId.getAddress() +
425 " this server is in the failed servers list");
426 }
427 IOException e = new FailedServerException(
428 "This server is in the failed servers list: " + remoteId.getAddress());
429 markClosed(e);
430 close();
431 throw e;
432 }
433
434 try {
435 if (LOG.isDebugEnabled()) {
436 LOG.debug("Connecting to "+remoteId);
437 }
438 setupConnection();
439 this.in = new DataInputStream(new BufferedInputStream
440 (new PingInputStream(NetUtils.getInputStream(socket))));
441 this.out = new DataOutputStream
442 (new BufferedOutputStream(NetUtils.getOutputStream(socket)));
443 writeHeader();
444
445
446 touch();
447
448
449 start();
450 } catch (Throwable t) {
451 failedServers.addToFailedServers(remoteId.address);
452 IOException e;
453 if (t instanceof IOException) {
454 e = (IOException)t;
455 } else {
456 e = new IOException("Could not set up IO Streams", t);
457 }
458 markClosed(e);
459 close();
460
461 throw e;
462 }
463 }
464
465 protected void closeConnection() {
466
467 if (socket != null) {
468 try {
469 socket.close();
470 } catch (IOException e) {
471 LOG.warn("Not able to close a socket", e);
472 }
473 }
474
475
476 socket = null;
477 }
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494 private void handleConnectionFailure(
495 int curRetries, int maxRetries, IOException ioe) throws IOException {
496
497 closeConnection();
498
499
500 if (curRetries >= maxRetries) {
501 throw ioe;
502 }
503
504
505 try {
506 Thread.sleep(failureSleep);
507 } catch (InterruptedException ignored) {}
508
509 LOG.info("Retrying connect to server: " + remoteId.getAddress() +
510 " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
511 " time(s).");
512 }
513
514
515
516
517 private void writeHeader() throws IOException {
518 out.write(HBaseServer.HEADER.array());
519 out.write(HBaseServer.CURRENT_VERSION);
520
521 DataOutputBuffer buf = new DataOutputBuffer();
522 header.write(buf);
523
524 int bufLen = buf.getLength();
525 out.writeInt(bufLen);
526 out.write(buf.getData(), 0, bufLen);
527 }
528
529
530
531
532
533
534
535 @SuppressWarnings({"ThrowableInstanceNeverThrown"})
536 protected synchronized boolean waitForWork() {
537 if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
538 long timeout = maxIdleTime-
539 (System.currentTimeMillis()-lastActivity.get());
540 if (timeout>0) {
541 try {
542 wait(timeout);
543 } catch (InterruptedException ignored) {}
544 }
545 }
546
547 if (!calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
548 return true;
549 } else if (shouldCloseConnection.get()) {
550 return false;
551 } else if (calls.isEmpty()) {
552 markClosed(null);
553 return false;
554 } else {
555 markClosed((IOException)new IOException().initCause(
556 new InterruptedException()));
557 return false;
558 }
559 }
560
561 public InetSocketAddress getRemoteAddress() {
562 return remoteId.getAddress();
563 }
564
565
566
567
568 protected synchronized void sendPing() throws IOException {
569 long curTime = System.currentTimeMillis();
570 if ( curTime - lastActivity.get() >= pingInterval) {
571 lastActivity.set(curTime);
572
573 synchronized (this.out) {
574 out.writeInt(PING_CALL_ID);
575 out.flush();
576 }
577 }
578 }
579
580 @Override
581 public void run() {
582 if (LOG.isDebugEnabled())
583 LOG.debug(getName() + ": starting, having connections "
584 + connections.size());
585
586 try {
587 while (waitForWork()) {
588 receiveResponse();
589 }
590 } catch (Throwable t) {
591 LOG.warn("Unexpected exception receiving call responses", t);
592 markClosed(new IOException("Unexpected exception receiving call responses", t));
593 }
594
595 close();
596
597 if (LOG.isDebugEnabled())
598 LOG.debug(getName() + ": stopped, remaining connections "
599 + connections.size());
600 }
601
602
603
604
605
606 protected void sendParam(Call call) {
607 if (shouldCloseConnection.get()) {
608 return;
609 }
610
611
612
613 final DataOutputBuffer d = new DataOutputBuffer();
614 try {
615 if (LOG.isDebugEnabled())
616 LOG.debug(getName() + " sending #" + call.id);
617
618 d.writeInt(0xdeadbeef);
619 d.writeInt(call.id);
620 call.param.write(d);
621 byte[] data = d.getData();
622 int dataLength = d.getLength();
623
624 Bytes.putInt(data, 0, dataLength - 4);
625
626 synchronized (this.out) {
627 out.write(data, 0, dataLength);
628 out.flush();
629 }
630 } catch(IOException e) {
631 markClosed(e);
632 } finally {
633
634
635 IOUtils.closeStream(d);
636 }
637 }
638
639
640
641
642 protected void receiveResponse() {
643 if (shouldCloseConnection.get()) {
644 return;
645 }
646 touch();
647
648 try {
649
650
651
652
653
654 int id = in.readInt();
655
656 if (LOG.isDebugEnabled())
657 LOG.debug(getName() + " got value #" + id);
658 Call call = calls.get(id);
659
660
661 byte flag = in.readByte();
662 boolean isError = ResponseFlag.isError(flag);
663 if (ResponseFlag.isLength(flag)) {
664
665 in.readInt();
666 }
667 int state = in.readInt();
668 if (isError) {
669 if (call != null) {
670
671 call.setException(new RemoteException(WritableUtils.readString(in),
672 WritableUtils.readString(in)));
673 }
674 } else {
675 Writable value = ReflectionUtils.newInstance(valueClass, conf);
676 value.readFields(in);
677
678
679 if (call != null) {
680 call.setValue(value);
681 }
682 }
683 calls.remove(id);
684 } catch (IOException e) {
685 if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
686
687
688
689 closeException = e;
690 } else {
691
692
693 markClosed(e);
694 }
695 } finally {
696 if (remoteId.rpcTimeout > 0) {
697 cleanupCalls(remoteId.rpcTimeout);
698 }
699 }
700 }
701
702 protected synchronized void markClosed(IOException e) {
703 if (shouldCloseConnection.compareAndSet(false, true)) {
704 closeException = e;
705 notifyAll();
706 }
707 }
708
709
710 protected synchronized void close() {
711 if (!shouldCloseConnection.get()) {
712 LOG.error("The connection is not in the closed state");
713 return;
714 }
715
716
717
718 synchronized (connections) {
719 connections.remove(remoteId, this);
720 }
721
722
723 IOUtils.closeStream(out);
724 IOUtils.closeStream(in);
725
726
727 if (closeException == null) {
728 if (!calls.isEmpty()) {
729 LOG.warn(
730 "A connection is closed for no cause and calls are not empty");
731
732
733 closeException = new IOException("Unexpected closed connection");
734 cleanupCalls();
735 }
736 } else {
737
738 if (LOG.isDebugEnabled()) {
739 LOG.debug("closing ipc connection to " + remoteId.address + ": " +
740 closeException.getMessage(),closeException);
741 }
742
743
744 cleanupCalls();
745 }
746 if (LOG.isDebugEnabled())
747 LOG.debug(getName() + ": closed");
748 }
749
750
751 protected void cleanupCalls() {
752 cleanupCalls(0);
753 }
754
755 protected void cleanupCalls(long rpcTimeout) {
756 Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
757 while (itor.hasNext()) {
758 Call c = itor.next().getValue();
759 long waitTime = System.currentTimeMillis() - c.getStartTime();
760 if (waitTime >= rpcTimeout) {
761 if (this.closeException == null) {
762
763
764
765
766
767
768 this.closeException = new CallTimeoutException("Call id=" + c.id +
769 ", waitTime=" + waitTime + ", rpcTimetout=" + rpcTimeout);
770 }
771 c.setException(this.closeException);
772 synchronized (c) {
773 c.notifyAll();
774 }
775 itor.remove();
776 } else {
777 break;
778 }
779 }
780 try {
781 if (!calls.isEmpty()) {
782 Call firstCall = calls.get(calls.firstKey());
783 long maxWaitTime = System.currentTimeMillis() - firstCall.getStartTime();
784 if (maxWaitTime < rpcTimeout) {
785 rpcTimeout -= maxWaitTime;
786 }
787 }
788 if (!shouldCloseConnection.get()) {
789 closeException = null;
790 if (socket != null) {
791 socket.setSoTimeout((int) rpcTimeout);
792 }
793 }
794 } catch (SocketException e) {
795 LOG.debug("Couldn't lower timeout, which may result in longer than expected calls");
796 }
797 }
798 }
799
800
801
802
803 public static class CallTimeoutException extends IOException {
804 public CallTimeoutException(final String msg) {
805 super(msg);
806 }
807 }
808
809
810 protected class ParallelCall extends Call {
811 private final ParallelResults results;
812 protected final int index;
813
814 public ParallelCall(Writable param, ParallelResults results, int index) {
815 super(param);
816 this.results = results;
817 this.index = index;
818 }
819
820
821 @Override
822 protected void callComplete() {
823 results.callComplete(this);
824 }
825 }
826
827
828 protected static class ParallelResults {
829 protected final Writable[] values;
830 protected int size;
831 protected int count;
832
833 public ParallelResults(int size) {
834 this.values = new Writable[size];
835 this.size = size;
836 }
837
838
839
840
841 synchronized void callComplete(ParallelCall call) {
842
843 values[call.index] = call.value;
844 count++;
845 if (count == size)
846 notify();
847 }
848 }
849
850
851
852
853
854
855
856
857 public HBaseClient(Class<? extends Writable> valueClass, Configuration conf,
858 SocketFactory factory) {
859 this.valueClass = valueClass;
860 this.maxIdleTime =
861 conf.getInt("hbase.ipc.client.connection.maxidletime", 10000);
862 this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
863 this.failureSleep = conf.getInt("hbase.client.pause", 1000);
864 this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", false);
865 this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
866 this.pingInterval = getPingInterval(conf);
867 if (LOG.isDebugEnabled()) {
868 LOG.debug("The ping interval is" + this.pingInterval + "ms.");
869 }
870 this.conf = conf;
871 this.socketFactory = factory;
872 this.clusterId = conf.get(HConstants.CLUSTER_ID, "default");
873 this.connections = new PoolMap<ConnectionId, Connection>(
874 getPoolType(conf), getPoolSize(conf));
875 String hostName = this.conf.get("hbase.regionserver.rpc.client.socket.bind.address");
876 if (hostName != null) {
877 this.bindAddress = new InetSocketAddress(hostName, 0);
878 } else {
879 this.bindAddress = null;
880 }
881 this.failedServers = new FailedServers(conf);
882 }
883
884
885
886
887
888
889 public HBaseClient(Class<? extends Writable> valueClass, Configuration conf) {
890 this(valueClass, conf, NetUtils.getDefaultSocketFactory(conf));
891 }
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908 protected static PoolType getPoolType(Configuration config) {
909 return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE),
910 PoolType.RoundRobin, PoolType.ThreadLocal);
911 }
912
913
914
915
916
917
918
919
920 protected static int getPoolSize(Configuration config) {
921 return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1);
922 }
923
924
925
926
927
928 SocketFactory getSocketFactory() {
929 return socketFactory;
930 }
931
932
933
934 public void stop() {
935 if (LOG.isDebugEnabled()) {
936 LOG.debug("Stopping client");
937 }
938
939 if (!running.compareAndSet(true, false)) {
940 return;
941 }
942
943
944 synchronized (connections) {
945 for (Connection conn : connections.values()) {
946 conn.interrupt();
947 }
948 }
949
950
951 while (!connections.isEmpty()) {
952 try {
953 Thread.sleep(100);
954 } catch (InterruptedException ignored) {
955 }
956 }
957 }
958
959
960
961
962
963
964
965
966
967 public Writable call(Writable param, InetSocketAddress address)
968 throws IOException, InterruptedException {
969 return call(param, address, null, 0);
970 }
971
972 public Writable call(Writable param, InetSocketAddress addr,
973 User ticket, int rpcTimeout)
974 throws IOException, InterruptedException {
975 return call(param, addr, null, ticket, rpcTimeout);
976 }
977
978
979
980
981
982
983 public Writable call(Writable param, InetSocketAddress addr,
984 Class<? extends VersionedProtocol> protocol,
985 User ticket, int rpcTimeout)
986 throws InterruptedException, IOException {
987 Call call = new Call(param);
988 Connection connection = getConnection(addr, protocol, ticket, rpcTimeout, call);
989 connection.sendParam(call);
990 boolean interrupted = false;
991
992 synchronized (call) {
993 while (!call.done) {
994 if (connection.shouldCloseConnection.get()) {
995 throw new IOException("Unexpected closed connection");
996 }
997 try {
998 call.wait(1000);
999 } catch (InterruptedException ignored) {
1000
1001 interrupted = true;
1002 }
1003 }
1004
1005 if (interrupted) {
1006
1007 Thread.currentThread().interrupt();
1008 }
1009
1010 if (call.error != null) {
1011 if (call.error instanceof RemoteException) {
1012 call.error.fillInStackTrace();
1013 throw call.error;
1014 }
1015
1016 throw wrapException(addr, call.error);
1017 }
1018 return call.value;
1019 }
1020 }
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034 @SuppressWarnings({"ThrowableInstanceNeverThrown"})
1035 protected IOException wrapException(InetSocketAddress addr,
1036 IOException exception) {
1037 if (exception instanceof ConnectException) {
1038
1039 return (ConnectException)new ConnectException(
1040 "Call to " + addr + " failed on connection exception: " + exception)
1041 .initCause(exception);
1042 } else if (exception instanceof SocketTimeoutException) {
1043 return (SocketTimeoutException)new SocketTimeoutException(
1044 "Call to " + addr + " failed on socket timeout exception: "
1045 + exception).initCause(exception);
1046 } else {
1047 return (IOException)new IOException(
1048 "Call to " + addr + " failed on local exception: " + exception)
1049 .initCause(exception);
1050
1051 }
1052 }
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064 @Deprecated
1065 public Writable[] call(Writable[] params, InetSocketAddress[] addresses)
1066 throws IOException, InterruptedException {
1067 return call(params, addresses, null, null);
1068 }
1069
1070
1071
1072
1073
1074 public Writable[] call(Writable[] params, InetSocketAddress[] addresses,
1075 Class<? extends VersionedProtocol> protocol,
1076 User ticket)
1077 throws IOException, InterruptedException {
1078 if (addresses.length == 0) return new Writable[0];
1079
1080 ParallelResults results = new ParallelResults(params.length);
1081
1082
1083 synchronized (results) {
1084 for (int i = 0; i < params.length; i++) {
1085 ParallelCall call = new ParallelCall(params[i], results, i);
1086 try {
1087 Connection connection =
1088 getConnection(addresses[i], protocol, ticket, 0, call);
1089 connection.sendParam(call);
1090 } catch (IOException e) {
1091
1092 LOG.info("Calling "+addresses[i]+" caught: " +
1093 e.getMessage(),e);
1094 results.size--;
1095 }
1096 }
1097 while (results.count != results.size) {
1098 try {
1099 results.wait();
1100 } catch (InterruptedException ignored) {}
1101 }
1102
1103 return results.values;
1104 }
1105 }
1106
1107
1108
1109 protected Connection getConnection(InetSocketAddress addr,
1110 Class<? extends VersionedProtocol> protocol,
1111 User ticket,
1112 int rpcTimeout,
1113 Call call)
1114 throws IOException, InterruptedException {
1115 if (!running.get()) {
1116
1117 throw new IOException("The client is stopped");
1118 }
1119 Connection connection;
1120
1121
1122
1123
1124 ConnectionId remoteId = new ConnectionId(addr, protocol, ticket, rpcTimeout);
1125 synchronized (connections) {
1126 connection = connections.get(remoteId);
1127 if (connection == null) {
1128 connection = createConnection(remoteId);
1129 connections.put(remoteId, connection);
1130 }
1131 }
1132 connection.addCall(call);
1133
1134
1135
1136
1137
1138
1139
1140
1141 connection.setupIOstreams();
1142 return connection;
1143 }
1144
1145
1146
1147
1148
1149 protected static class ConnectionId {
1150 final InetSocketAddress address;
1151 final User ticket;
1152 final int rpcTimeout;
1153 Class<? extends VersionedProtocol> protocol;
1154 private static final int PRIME = 16777619;
1155
1156 ConnectionId(InetSocketAddress address,
1157 Class<? extends VersionedProtocol> protocol,
1158 User ticket,
1159 int rpcTimeout) {
1160 this.protocol = protocol;
1161 this.address = address;
1162 this.ticket = ticket;
1163 this.rpcTimeout = rpcTimeout;
1164 }
1165
1166 InetSocketAddress getAddress() {
1167 return address;
1168 }
1169
1170 Class<? extends VersionedProtocol> getProtocol() {
1171 return protocol;
1172 }
1173
1174 User getTicket() {
1175 return ticket;
1176 }
1177
1178 @Override
1179 public boolean equals(Object obj) {
1180 if (obj instanceof ConnectionId) {
1181 ConnectionId id = (ConnectionId) obj;
1182 return address.equals(id.address) && protocol == id.protocol &&
1183 ((ticket != null && ticket.equals(id.ticket)) ||
1184 (ticket == id.ticket)) && rpcTimeout == id.rpcTimeout;
1185 }
1186 return false;
1187 }
1188
1189 @Override
1190 public int hashCode() {
1191 return (address.hashCode() + PRIME * (
1192 PRIME * System.identityHashCode(protocol) ^
1193 (ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout;
1194 }
1195 }
1196
1197
1198
1199
1200 public String getClusterId() {
1201 return clusterId;
1202 }
1203 }