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