1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.ipc;
19
20 import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader;
21 import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException;
22 import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited;
23 import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
24 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
25 import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
26
27 import com.google.protobuf.Message;
28 import com.google.protobuf.Message.Builder;
29 import com.google.protobuf.RpcCallback;
30
31 import java.io.BufferedInputStream;
32 import java.io.BufferedOutputStream;
33 import java.io.DataInputStream;
34 import java.io.DataOutputStream;
35 import java.io.IOException;
36 import java.io.InputStream;
37 import java.io.InterruptedIOException;
38 import java.io.OutputStream;
39 import java.net.InetSocketAddress;
40 import java.net.Socket;
41 import java.net.SocketTimeoutException;
42 import java.net.UnknownHostException;
43 import java.nio.ByteBuffer;
44 import java.security.PrivilegedExceptionAction;
45 import java.util.ArrayDeque;
46 import java.util.Locale;
47 import java.util.Queue;
48 import java.util.concurrent.ConcurrentHashMap;
49 import java.util.concurrent.ConcurrentMap;
50 import java.util.concurrent.ThreadLocalRandom;
51
52 import javax.security.sasl.SaslException;
53
54 import org.apache.commons.logging.Log;
55 import org.apache.commons.logging.LogFactory;
56 import org.apache.hadoop.conf.Configuration;
57 import org.apache.hadoop.hbase.CellScanner;
58 import org.apache.hadoop.hbase.DoNotRetryIOException;
59 import org.apache.hadoop.hbase.classification.InterfaceAudience;
60 import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
61 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
62 import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
63 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
64 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
65 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
66 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
67 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
68 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
69 import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
70 import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
71 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
72 import org.apache.hadoop.hbase.util.ExceptionUtil;
73 import org.apache.hadoop.io.IOUtils;
74 import org.apache.hadoop.ipc.RemoteException;
75 import org.apache.hadoop.net.NetUtils;
76 import org.apache.hadoop.security.UserGroupInformation;
77 import org.apache.htrace.Trace;
78 import org.apache.htrace.TraceScope;
79
80
81
82
83
84
85 @InterfaceAudience.Private
86 class BlockingRpcConnection extends RpcConnection implements Runnable {
87
88 private static final Log LOG = LogFactory.getLog(BlockingRpcConnection.class);
89
90 private final BlockingRpcClient rpcClient;
91
92 private final String threadName;
93 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
94 justification = "We are always under lock actually")
95 private Thread thread;
96
97
98 protected Socket socket = null;
99 private DataInputStream in;
100 private DataOutputStream out;
101
102 private HBaseSaslRpcClient saslRpcClient;
103
104
105 private final ConcurrentMap<Integer, Call> calls = new ConcurrentHashMap<>();
106
107 private final CallSender callSender;
108
109 private boolean closed = false;
110
111 private byte[] connectionHeaderPreamble;
112
113 private byte[] connectionHeaderWithLength;
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132 private class CallSender extends Thread {
133
134 private final Queue<Call> callsToWrite;
135
136 private final int maxQueueSize;
137
138 public CallSender(String name, Configuration conf) {
139 int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
140 callsToWrite = new ArrayDeque<>(queueSize);
141 this.maxQueueSize = queueSize;
142 setDaemon(true);
143 setName(name + " - writer");
144 }
145
146 public void sendCall(final Call call) throws IOException {
147 if (callsToWrite.size() >= maxQueueSize) {
148 throw new IOException("Can't add the call " + call.id +
149 " to the write queue. callsToWrite.size()=" + callsToWrite.size());
150 }
151 callsToWrite.offer(call);
152 BlockingRpcConnection.this.notifyAll();
153 }
154
155 public void remove(Call call) {
156 callsToWrite.remove(call);
157
158
159 calls.remove(call.id);
160 call.setException(new CallCancelledException("Call id=" + call.id + ", waitTime=" +
161 (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout=" +
162 call.timeout));
163 }
164
165
166
167
168 @Override
169 public void run() {
170 synchronized (BlockingRpcConnection.this) {
171 while (!closed) {
172 if (callsToWrite.isEmpty()) {
173
174
175
176 try {
177 BlockingRpcConnection.this.wait();
178 } catch (InterruptedException e) {
179 }
180
181 continue;
182 }
183 Call call = callsToWrite.poll();
184 if (call.isDone()) {
185 continue;
186 }
187 try {
188 tracedWriteRequest(call);
189 } catch (IOException e) {
190
191
192 if (LOG.isDebugEnabled()) {
193 LOG.debug("call write error for call #" + call.id, e);
194 }
195 call.setException(e);
196 closeConn(e);
197 }
198 }
199 }
200 }
201
202
203
204
205 public void cleanup(IOException e) {
206 IOException ie =
207 new ConnectionClosingException("Connection to " + remoteId.getAddress() + " is closing.");
208 for (Call call : callsToWrite) {
209 call.setException(ie);
210 }
211 callsToWrite.clear();
212 }
213 }
214
215 BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException {
216 super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
217 rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor,
218 rpcClient.metrics);
219 this.rpcClient = rpcClient;
220 this.connectionHeaderPreamble = getConnectionHeaderPreamble();
221 ConnectionHeader header = getConnectionHeader();
222 ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize());
223 DataOutputStream dos = new DataOutputStream(baos);
224 dos.writeInt(header.getSerializedSize());
225 header.writeTo(dos);
226 assert baos.size() == 4 + header.getSerializedSize();
227 this.connectionHeaderWithLength = baos.getBuffer();
228
229 UserGroupInformation ticket = remoteId.ticket.getUGI();
230 this.threadName = "IPC Client (" + this.rpcClient.socketFactory.hashCode() +
231 ") connection to " + remoteId.getAddress().toString() +
232 ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName()));
233
234 if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) {
235 callSender = new CallSender(threadName, this.rpcClient.conf);
236 callSender.start();
237 } else {
238 callSender = null;
239 }
240 }
241
242
243 protected void setupConnection() throws IOException {
244 short ioFailures = 0;
245 short timeoutFailures = 0;
246 while (true) {
247 try {
248 this.socket = this.rpcClient.socketFactory.createSocket();
249 this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay());
250 this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive);
251 if (this.rpcClient.localAddr != null) {
252 this.socket.bind(this.rpcClient.localAddr);
253 }
254 if (this.rpcClient.metrics != null) {
255 this.rpcClient.metrics.incrNsLookups();
256 }
257 InetSocketAddress remoteAddr = remoteId.getAddress().toSocketAddress();
258 if (remoteAddr.isUnresolved()) {
259 if (this.rpcClient.metrics != null) {
260 this.rpcClient.metrics.incrNsLookupsFailed();
261 }
262 throw new UnknownHostException(remoteId.getAddress() + " could not be resolved");
263 }
264 NetUtils.connect(this.socket, remoteAddr, this.rpcClient.connectTO);
265 this.socket.setSoTimeout(this.rpcClient.readTO);
266 return;
267 } catch (SocketTimeoutException toe) {
268
269
270
271 handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe);
272 } catch (IOException ie) {
273 handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie);
274 }
275 }
276 }
277
278
279
280
281
282
283
284
285
286
287
288 private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
289 throws IOException {
290 closeSocket();
291
292
293 if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
294 throw ioe;
295 }
296
297
298 try {
299 Thread.sleep(this.rpcClient.failureSleep);
300 } catch (InterruptedException ie) {
301 ExceptionUtil.rethrowIfInterrupt(ie);
302 }
303
304 LOG.info("Retrying connect to server: " + remoteId.getAddress() + " after sleeping " +
305 this.rpcClient.failureSleep + "ms. Already tried " + curRetries + " time(s).");
306 }
307
308
309
310
311
312
313 private synchronized boolean waitForWork() {
314
315
316 long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose;
317 for (;;) {
318 if (thread == null) {
319 return false;
320 }
321 if (!calls.isEmpty()) {
322 return true;
323 }
324 if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
325 closeConn(
326 new IOException("idle connection closed with " + calls.size() + " pending request(s)"));
327 return false;
328 }
329 try {
330 wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000));
331 } catch (InterruptedException e) {
332 }
333 }
334 }
335
336 @Override
337 public void run() {
338 if (LOG.isTraceEnabled()) {
339 LOG.trace(threadName + ": starting, connections " + this.rpcClient.connections.size());
340 }
341 while (waitForWork()) {
342 readResponse();
343 }
344 if (LOG.isTraceEnabled()) {
345 LOG.trace(threadName + ": stopped, connections " + this.rpcClient.connections.size());
346 }
347 }
348
349 private void disposeSasl() {
350 if (saslRpcClient != null) {
351 saslRpcClient.dispose();
352 saslRpcClient = null;
353 }
354 }
355
356 private boolean setupSaslConnection(final InputStream in2, final OutputStream out2)
357 throws IOException {
358 saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal,
359 this.rpcClient.fallbackAllowed, this.rpcClient.conf.get("hbase.rpc.protection",
360 QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
361 return saslRpcClient.saslConnect(in2, out2);
362 }
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378 private void handleSaslConnectionFailure(final int currRetries, final int maxRetries,
379 final Exception ex, final UserGroupInformation user)
380 throws IOException, InterruptedException {
381 closeSocket();
382 user.doAs(new PrivilegedExceptionAction<Object>() {
383 @Override
384 public Object run() throws IOException, InterruptedException {
385 if (shouldAuthenticateOverKrb()) {
386 if (currRetries < maxRetries) {
387 if (LOG.isDebugEnabled()) {
388 LOG.debug("Exception encountered while connecting to " + "the server : " + ex);
389 }
390
391 relogin();
392 disposeSasl();
393
394
395
396
397 Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1);
398 return null;
399 } else {
400 String msg = "Couldn't setup connection for " +
401 UserGroupInformation.getLoginUser().getUserName() + " to " + serverPrincipal;
402 LOG.warn(msg, ex);
403 throw new IOException(msg, ex);
404 }
405 } else {
406 LOG.warn("Exception encountered while connecting to " + "the server : " + ex);
407 }
408 if (ex instanceof RemoteException) {
409 throw (RemoteException) ex;
410 }
411 if (ex instanceof SaslException) {
412 String msg = "SASL authentication failed." +
413 " The most likely cause is missing or invalid credentials." + " Consider 'kinit'.";
414 LOG.fatal(msg, ex);
415 throw new RuntimeException(msg, ex);
416 }
417 throw new IOException(ex);
418 }
419 });
420 }
421
422 private void setupIOstreams() throws IOException {
423 if (socket != null) {
424
425 return;
426 }
427
428 if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) {
429 if (LOG.isDebugEnabled()) {
430 LOG.debug("Not trying to connect to " + remoteId.getAddress() +
431 " this server is in the failed servers list");
432 }
433 throw new FailedServerException(
434 "This server is in the failed servers list: " + remoteId.getAddress());
435 }
436
437 try {
438 if (LOG.isDebugEnabled()) {
439 LOG.debug("Connecting to " + remoteId.getAddress());
440 }
441
442 short numRetries = 0;
443 int reloginMaxRetries = this.rpcClient.conf.getInt("hbase.security.relogin.maxretries", 5);
444 while (true) {
445 setupConnection();
446 InputStream inStream = NetUtils.getInputStream(socket);
447
448 OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO);
449
450 writeConnectionHeaderPreamble(outStream);
451 if (useSasl) {
452 final InputStream in2 = inStream;
453 final OutputStream out2 = outStream;
454 UserGroupInformation ticket = getUGI();
455 boolean continueSasl;
456 if (ticket == null) {
457 throw new FatalConnectionException("ticket/user is null");
458 }
459 try {
460 continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
461 @Override
462 public Boolean run() throws IOException {
463 return setupSaslConnection(in2, out2);
464 }
465 });
466 } catch (Exception ex) {
467 ExceptionUtil.rethrowIfInterrupt(ex);
468 handleSaslConnectionFailure(numRetries++, reloginMaxRetries, ex, ticket);
469 continue;
470 }
471 if (continueSasl) {
472
473 inStream = saslRpcClient.getInputStream(inStream);
474 outStream = saslRpcClient.getOutputStream(outStream);
475 } else {
476
477
478
479 }
480 }
481 this.in = new DataInputStream(new BufferedInputStream(inStream));
482 this.out = new DataOutputStream(new BufferedOutputStream(outStream));
483
484 writeConnectionHeader();
485 break;
486 }
487 } catch (Throwable t) {
488 closeSocket();
489 IOException e = ExceptionUtil.asInterrupt(t);
490 if (e == null) {
491 this.rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), t);
492 if (t instanceof LinkageError) {
493
494 e = new DoNotRetryIOException(t);
495 } else if (t instanceof IOException) {
496 e = (IOException) t;
497 } else {
498 e = new IOException("Could not set up IO Streams to " + remoteId.getAddress(), t);
499 }
500 }
501 throw e;
502 }
503
504
505 thread = new Thread(this, threadName);
506 thread.setDaemon(true);
507 thread.start();
508 }
509
510
511
512
513 private void writeConnectionHeaderPreamble(OutputStream out) throws IOException {
514 out.write(connectionHeaderPreamble);
515 out.flush();
516 }
517
518
519
520
521 private void writeConnectionHeader() throws IOException {
522 this.out.write(connectionHeaderWithLength);
523 this.out.flush();
524 }
525
526 private void tracedWriteRequest(Call call) throws IOException {
527 try (TraceScope ignored = Trace.startSpan("RpcClientImpl.tracedWriteRequest", call.span)) {
528 writeRequest(call);
529 }
530 }
531
532
533
534
535
536
537 private void writeRequest(Call call) throws IOException {
538 ByteBuffer cellBlock =
539 this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec, this.compressor, call.cells);
540 CellBlockMeta cellBlockMeta;
541 if (cellBlock != null) {
542 cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.limit()).build();
543 } else {
544 cellBlockMeta = null;
545 }
546 RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta);
547
548 setupIOstreams();
549
550
551
552
553 if (Thread.interrupted()) {
554 throw new InterruptedIOException();
555 }
556
557 calls.put(call.id, call);
558
559
560 try {
561 call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock));
562 } catch (Throwable t) {
563 if(LOG.isTraceEnabled()) {
564 LOG.trace("Error while writing call, call_id:" + call.id, t);
565 }
566 IOException e = IPCUtil.toIOE(t);
567 closeConn(e);
568 return;
569 }
570 notifyAll();
571 }
572
573
574
575
576 private void readResponse() {
577 Call call = null;
578 boolean expectedCall = false;
579 try {
580
581
582 int totalSize = in.readInt();
583
584
585 ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
586 int id = responseHeader.getCallId();
587 call = calls.remove(id);
588 expectedCall = (call != null && !call.isDone());
589 if (!expectedCall) {
590
591
592
593
594
595 int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader);
596 int whatIsLeftToRead = totalSize - readSoFar;
597 IOUtils.skipFully(in, whatIsLeftToRead);
598 if (call != null) {
599 call.callStats.setResponseSizeBytes(totalSize);
600 call.callStats
601 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
602 }
603 return;
604 }
605 if (responseHeader.hasException()) {
606 ExceptionResponse exceptionResponse = responseHeader.getException();
607 RemoteException re = createRemoteException(exceptionResponse);
608 call.setException(re);
609 call.callStats.setResponseSizeBytes(totalSize);
610 call.callStats
611 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
612 if (isFatalConnectionException(exceptionResponse)) {
613 synchronized (this) {
614 closeConn(re);
615 }
616 }
617 } else {
618 Message value = null;
619 if (call.responseDefaultType != null) {
620 Builder builder = call.responseDefaultType.newBuilderForType();
621 ProtobufUtil.mergeDelimitedFrom(builder, in);
622 value = builder.build();
623 }
624 CellScanner cellBlockScanner = null;
625 if (responseHeader.hasCellBlockMeta()) {
626 int size = responseHeader.getCellBlockMeta().getLength();
627 byte[] cellBlock = new byte[size];
628 IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
629 cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec,
630 this.compressor, cellBlock);
631 }
632 call.setResponse(value, cellBlockScanner);
633 call.callStats.setResponseSizeBytes(totalSize);
634 call.callStats
635 .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());
636 }
637 } catch (IOException e) {
638 if (expectedCall) {
639 call.setException(e);
640 }
641 if (e instanceof SocketTimeoutException) {
642
643
644
645 if (LOG.isTraceEnabled()) {
646 LOG.trace("ignored", e);
647 }
648 } else {
649 synchronized (this) {
650 closeConn(e);
651 }
652 }
653 }
654 }
655
656 @Override
657 protected synchronized void callTimeout(Call call) {
658
659 calls.remove(call.id);
660 }
661
662
663 private void closeSocket() {
664 IOUtils.closeStream(out);
665 IOUtils.closeStream(in);
666 IOUtils.closeSocket(socket);
667 out = null;
668 in = null;
669 socket = null;
670 }
671
672
673 private void closeConn(IOException e) {
674 if (thread == null) {
675 return;
676 }
677 thread.interrupt();
678 thread = null;
679 closeSocket();
680 if (callSender != null) {
681 callSender.cleanup(e);
682 }
683 for (Call call : calls.values()) {
684 call.setException(e);
685 }
686 calls.clear();
687 }
688
689
690 @Override
691 public synchronized void shutdown() {
692 closed = true;
693 if (callSender != null) {
694 callSender.interrupt();
695 }
696 closeConn(new IOException("connection to " + remoteId.getAddress() + " closed"));
697 }
698
699 @Override
700 public void cleanupConnection() {
701
702 }
703
704 @Override
705 public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
706 throws IOException {
707 pcrc.notifyOnCancel(new RpcCallback<Object>() {
708
709 @Override
710 public void run(Object parameter) {
711 setCancelled(call);
712 synchronized (BlockingRpcConnection.this) {
713 if (callSender != null) {
714 callSender.remove(call);
715 } else {
716 calls.remove(call.id);
717 }
718 }
719 }
720 }, new CancellationCallback() {
721
722 @Override
723 public void run(boolean cancelled) throws IOException {
724 if (cancelled) {
725 setCancelled(call);
726 return;
727 }
728 scheduleTimeoutTask(call);
729 if (callSender != null) {
730 callSender.sendCall(call);
731 } else {
732 tracedWriteRequest(call);
733 }
734 }
735 });
736 }
737
738 @Override
739 public synchronized boolean isActive() {
740 return thread != null;
741 }
742 }