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