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