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