1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.ipc;
20
21 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION;
22
23 import java.io.ByteArrayInputStream;
24 import java.io.ByteArrayOutputStream;
25 import java.io.DataOutputStream;
26 import java.io.IOException;
27 import java.net.BindException;
28 import java.net.InetAddress;
29 import java.net.InetSocketAddress;
30 import java.net.ServerSocket;
31 import java.net.Socket;
32 import java.net.SocketException;
33 import java.net.UnknownHostException;
34 import java.nio.ByteBuffer;
35 import java.nio.channels.CancelledKeyException;
36 import java.nio.channels.Channels;
37 import java.nio.channels.ClosedChannelException;
38 import java.nio.channels.GatheringByteChannel;
39 import java.nio.channels.ReadableByteChannel;
40 import java.nio.channels.SelectionKey;
41 import java.nio.channels.Selector;
42 import java.nio.channels.ServerSocketChannel;
43 import java.nio.channels.SocketChannel;
44 import java.nio.channels.WritableByteChannel;
45 import java.security.PrivilegedExceptionAction;
46 import java.util.ArrayList;
47 import java.util.Collections;
48 import java.util.HashMap;
49 import java.util.Iterator;
50 import java.util.LinkedList;
51 import java.util.List;
52 import java.util.Map;
53 import java.util.Random;
54 import java.util.concurrent.ExecutorService;
55 import java.util.concurrent.Executors;
56 import java.util.concurrent.atomic.AtomicInteger;
57
58 import javax.security.sasl.Sasl;
59 import javax.security.sasl.SaslException;
60 import javax.security.sasl.SaslServer;
61
62 import org.apache.commons.logging.Log;
63 import org.apache.commons.logging.LogFactory;
64 import org.apache.hadoop.hbase.classification.InterfaceAudience;
65 import org.apache.hadoop.hbase.classification.InterfaceStability;
66 import org.apache.hadoop.conf.Configuration;
67 import org.apache.hadoop.hbase.CellScanner;
68 import org.apache.hadoop.hbase.DoNotRetryIOException;
69 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
70 import org.apache.hadoop.hbase.HConstants;
71 import org.apache.hadoop.hbase.HRegionInfo;
72 import org.apache.hadoop.hbase.Server;
73 import org.apache.hadoop.hbase.TableName;
74 import org.apache.hadoop.hbase.client.Operation;
75 import org.apache.hadoop.hbase.codec.Codec;
76 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
77 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
78 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
79 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
80 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
81 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
82 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
83 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
84 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
85 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
86 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
87 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
88 import org.apache.hadoop.hbase.regionserver.HRegionServer;
89 import org.apache.hadoop.hbase.security.AccessDeniedException;
90 import org.apache.hadoop.hbase.security.AuthMethod;
91 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
92 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
93 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler;
94 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
95 import org.apache.hadoop.hbase.security.SaslStatus;
96 import org.apache.hadoop.hbase.security.SaslUtil;
97 import org.apache.hadoop.hbase.security.User;
98 import org.apache.hadoop.hbase.security.UserProvider;
99 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
100 import org.apache.hadoop.hbase.util.Bytes;
101 import org.apache.hadoop.hbase.util.Pair;
102 import org.apache.hadoop.io.BytesWritable;
103 import org.apache.hadoop.io.IntWritable;
104 import org.apache.hadoop.io.Writable;
105 import org.apache.hadoop.io.WritableUtils;
106 import org.apache.hadoop.io.compress.CompressionCodec;
107 import org.apache.hadoop.security.UserGroupInformation;
108 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
109 import org.apache.hadoop.security.authorize.AuthorizationException;
110 import org.apache.hadoop.security.authorize.PolicyProvider;
111 import org.apache.hadoop.security.authorize.ProxyUsers;
112 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
113 import org.apache.hadoop.security.token.SecretManager;
114 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
115 import org.apache.hadoop.security.token.TokenIdentifier;
116 import org.apache.hadoop.util.StringUtils;
117 import org.cliffc.high_scale_lib.Counter;
118 import org.cloudera.htrace.TraceInfo;
119 import org.codehaus.jackson.map.ObjectMapper;
120
121 import com.google.common.util.concurrent.ThreadFactoryBuilder;
122 import com.google.protobuf.BlockingService;
123 import com.google.protobuf.CodedInputStream;
124 import com.google.protobuf.Descriptors.MethodDescriptor;
125 import com.google.protobuf.Message;
126 import com.google.protobuf.ServiceException;
127 import com.google.protobuf.TextFormat;
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
151 @InterfaceStability.Evolving
152 public class RpcServer implements RpcServerInterface {
153
154
155 public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RpcServer");
156 private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
157 = new CallQueueTooBigException();
158
159 private final boolean authorize;
160 private boolean isSecurityEnabled;
161
162 public static final byte CURRENT_VERSION = 0;
163
164
165
166
167 public static final String FALLBACK_TO_INSECURE_CLIENT_AUTH =
168 "hbase.ipc.server.fallback-to-simple-auth-allowed";
169
170
171
172
173 static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
174
175
176
177
178 private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
179
180 static final int BUFFER_INITIAL_SIZE = 1024;
181
182 private static final String WARN_DELAYED_CALLS = "hbase.ipc.warn.delayedrpc.number";
183
184 private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
185
186 private final int warnDelayedCalls;
187
188 private AtomicInteger delayedCalls;
189 private final IPCUtil ipcUtil;
190
191 private static final String AUTH_FAILED_FOR = "Auth failed for ";
192 private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";
193 private static final Log AUDITLOG = LogFactory.getLog("SecurityLogger." +
194 Server.class.getName());
195 protected SecretManager<TokenIdentifier> secretManager;
196 protected ServiceAuthorizationManager authManager;
197
198
199
200
201 protected static final ThreadLocal<Call> CurCall = new ThreadLocal<Call>();
202
203
204 static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC
205 = new ThreadLocal<MonitoredRPCHandler>();
206
207 protected final InetSocketAddress isa;
208 protected int port;
209 private int readThreads;
210 protected int maxIdleTime;
211
212
213 protected int thresholdIdleConnections;
214
215
216
217 int maxConnectionsToNuke;
218
219
220
221 protected MetricsHBaseServer metrics;
222
223 protected final Configuration conf;
224
225 private int maxQueueSize;
226 protected int socketSendBufferSize;
227 protected final boolean tcpNoDelay;
228 protected final boolean tcpKeepAlive;
229 protected final long purgeTimeout;
230
231
232
233
234
235
236 volatile boolean running = true;
237
238
239
240
241
242 volatile boolean started = false;
243
244
245
246
247 protected final Counter callQueueSize = new Counter();
248
249 protected final List<Connection> connectionList =
250 Collections.synchronizedList(new LinkedList<Connection>());
251
252
253 private Listener listener = null;
254 protected Responder responder = null;
255 protected AuthenticationTokenSecretManager authTokenSecretMgr = null;
256 protected int numConnections = 0;
257
258 protected HBaseRPCErrorHandler errorHandler = null;
259
260 private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
261 private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
262
263
264 private static final int DEFAULT_WARN_RESPONSE_TIME = 10000;
265 private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
266
267 private static final ObjectMapper MAPPER = new ObjectMapper();
268
269 private final int warnResponseTime;
270 private final int warnResponseSize;
271 private final Object serverInstance;
272 private final List<BlockingServiceAndInterface> services;
273
274 private final RpcScheduler scheduler;
275
276 private UserProvider userProvider;
277
278 private final BoundedByteBufferPool reservoir;
279
280 private boolean allowFallbackToSimpleAuth;
281
282
283
284
285
286 class Call implements RpcCallContext {
287 protected int id;
288 protected BlockingService service;
289 protected MethodDescriptor md;
290 protected RequestHeader header;
291 protected Message param;
292
293 protected CellScanner cellScanner;
294 protected Connection connection;
295 protected long timestamp;
296
297
298
299
300 protected BufferChain response;
301 protected boolean delayResponse;
302 protected Responder responder;
303 protected boolean delayReturnValue;
304
305 protected long size;
306 protected boolean isError;
307 protected TraceInfo tinfo;
308 private ByteBuffer cellBlock = null;
309
310 private User user;
311 private InetAddress remoteAddress;
312
313
314
315
316 @Deprecated
317 Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
318 Message param, CellScanner cellScanner, Connection connection, Responder responder,
319 long size, TraceInfo tinfo) {
320 this(id, service, md, header, param, cellScanner, connection, responder, size, tinfo, null);
321 }
322
323 Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
324 Message param, CellScanner cellScanner, Connection connection, Responder responder,
325 long size, TraceInfo tinfo, InetAddress remoteAddress) {
326 this.id = id;
327 this.service = service;
328 this.md = md;
329 this.header = header;
330 this.param = param;
331 this.cellScanner = cellScanner;
332 this.connection = connection;
333 this.timestamp = System.currentTimeMillis();
334 this.response = null;
335 this.delayResponse = false;
336 this.responder = responder;
337 this.isError = false;
338 this.size = size;
339 this.tinfo = tinfo;
340 if (connection != null && connection.user != null) {
341 this.user = userProvider.create(connection.user);
342 } else {
343 this.user = null;
344 }
345 this.remoteAddress = remoteAddress;
346 }
347
348
349
350
351
352 void done() {
353 if (this.cellBlock != null) {
354
355 reservoir.putBuffer(this.cellBlock);
356 this.cellBlock = null;
357 }
358 }
359
360 @Override
361 public String toString() {
362 return toShortString() + " param: " +
363 (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") +
364 " connection: " + connection.toString();
365 }
366
367 protected RequestHeader getHeader() {
368 return this.header;
369 }
370
371 @Override
372 public User getRequestUser() {
373 return user;
374 }
375
376 @Override
377 public String getRequestUserName() {
378 User user = getRequestUser();
379 return user == null? null: user.getShortName();
380 }
381
382 @Override
383 public InetAddress getRemoteAddress() {
384 return remoteAddress;
385 }
386
387
388
389
390
391 String toShortString() {
392 String serviceName = this.connection.service != null?
393 this.connection.service.getDescriptorForType().getName() : "null";
394 StringBuilder sb = new StringBuilder();
395 sb.append("callId: ");
396 sb.append(this.id);
397 sb.append(" service: ");
398 sb.append(serviceName);
399 sb.append(" methodName: ");
400 sb.append((this.md != null) ? this.md.getName() : "");
401 sb.append(" size: ");
402 sb.append(StringUtils.humanReadableInt(this.size));
403 sb.append(" connection: ");
404 sb.append(connection.toString());
405 return sb.toString();
406 }
407
408 String toTraceString() {
409 String serviceName = this.connection.service != null ?
410 this.connection.service.getDescriptorForType().getName() : "";
411 String methodName = (this.md != null) ? this.md.getName() : "";
412 String result = serviceName + "." + methodName;
413 return result;
414 }
415
416 protected synchronized void setSaslTokenResponse(ByteBuffer response) {
417 this.response = new BufferChain(response);
418 }
419
420 protected synchronized void setResponse(Object m, final CellScanner cells,
421 Throwable t, String errorMsg) {
422 if (this.isError) return;
423 if (t != null) this.isError = true;
424 BufferChain bc = null;
425 try {
426 ResponseHeader.Builder headerBuilder = ResponseHeader.newBuilder();
427
428 Message result = (Message)m;
429
430 headerBuilder.setCallId(this.id);
431 if (t != null) {
432 ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
433 exceptionBuilder.setExceptionClassName(t.getClass().getName());
434 exceptionBuilder.setStackTrace(errorMsg);
435 exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException);
436 if (t instanceof RegionMovedException) {
437
438
439
440 RegionMovedException rme = (RegionMovedException)t;
441 exceptionBuilder.setHostname(rme.getHostname());
442 exceptionBuilder.setPort(rme.getPort());
443 }
444
445 headerBuilder.setException(exceptionBuilder.build());
446 }
447
448
449
450 this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec,
451 this.connection.compressionCodec, cells, reservoir);
452 if (this.cellBlock != null) {
453 CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
454
455 cellBlockBuilder.setLength(this.cellBlock.limit());
456 headerBuilder.setCellBlockMeta(cellBlockBuilder.build());
457 }
458 Message header = headerBuilder.build();
459
460
461
462 ByteBuffer bbHeader = IPCUtil.getDelimitedMessageAsByteBuffer(header);
463 ByteBuffer bbResult = IPCUtil.getDelimitedMessageAsByteBuffer(result);
464 int totalSize = bbHeader.capacity() + (bbResult == null? 0: bbResult.limit()) +
465 (this.cellBlock == null? 0: this.cellBlock.limit());
466 ByteBuffer bbTotalSize = ByteBuffer.wrap(Bytes.toBytes(totalSize));
467 bc = new BufferChain(bbTotalSize, bbHeader, bbResult, this.cellBlock);
468 if (connection.useWrap) {
469 bc = wrapWithSasl(bc);
470 }
471 } catch (IOException e) {
472 LOG.warn("Exception while creating response " + e);
473 }
474 this.response = bc;
475 }
476
477 private BufferChain wrapWithSasl(BufferChain bc)
478 throws IOException {
479 if (bc == null) return bc;
480 if (!this.connection.useSasl) return bc;
481
482
483 byte [] responseBytes = bc.getBytes();
484 byte [] token;
485
486
487 synchronized (connection.saslServer) {
488 token = connection.saslServer.wrap(responseBytes, 0, responseBytes.length);
489 }
490 if (LOG.isDebugEnabled())
491 LOG.debug("Adding saslServer wrapped token of size " + token.length
492 + " as call response.");
493
494 ByteBuffer bbTokenLength = ByteBuffer.wrap(Bytes.toBytes(token.length));
495 ByteBuffer bbTokenBytes = ByteBuffer.wrap(token);
496 return new BufferChain(bbTokenLength, bbTokenBytes);
497 }
498
499 @Override
500 public synchronized void endDelay(Object result) throws IOException {
501 assert this.delayResponse;
502 assert this.delayReturnValue || result == null;
503 this.delayResponse = false;
504 delayedCalls.decrementAndGet();
505 if (this.delayReturnValue) {
506 this.setResponse(result, null, null, null);
507 }
508 this.responder.doRespond(this);
509 }
510
511 @Override
512 public synchronized void endDelay() throws IOException {
513 this.endDelay(null);
514 }
515
516 @Override
517 public synchronized void startDelay(boolean delayReturnValue) {
518 assert !this.delayResponse;
519 this.delayResponse = true;
520 this.delayReturnValue = delayReturnValue;
521 int numDelayed = delayedCalls.incrementAndGet();
522 if (numDelayed > warnDelayedCalls) {
523 LOG.warn("Too many delayed calls: limit " + warnDelayedCalls + " current " + numDelayed);
524 }
525 }
526
527 @Override
528 public synchronized void endDelayThrowing(Throwable t) throws IOException {
529 this.setResponse(null, null, t, StringUtils.stringifyException(t));
530 this.delayResponse = false;
531 this.sendResponseIfReady();
532 }
533
534 @Override
535 public synchronized boolean isDelayed() {
536 return this.delayResponse;
537 }
538
539 @Override
540 public synchronized boolean isReturnValueDelayed() {
541 return this.delayReturnValue;
542 }
543
544 @Override
545 public boolean isClientCellBlockSupport() {
546 return this.connection != null && this.connection.codec != null;
547 }
548
549 @Override
550 public long disconnectSince() {
551 if (!connection.channel.isOpen()) {
552 return System.currentTimeMillis() - timestamp;
553 } else {
554 return -1L;
555 }
556 }
557
558 public long getSize() {
559 return this.size;
560 }
561
562
563
564
565
566
567 public synchronized void sendResponseIfReady() throws IOException {
568 if (!this.delayResponse) {
569 this.responder.doRespond(this);
570 }
571 }
572 }
573
574
575 private class Listener extends Thread {
576
577 private ServerSocketChannel acceptChannel = null;
578 private Selector selector = null;
579 private Reader[] readers = null;
580 private int currentReader = 0;
581 private Random rand = new Random();
582 private long lastCleanupRunTime = 0;
583
584 private long cleanupInterval = 10000;
585
586 private int backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size",
587 conf.getInt("ipc.server.listen.queue.size", 128));
588
589 private ExecutorService readPool;
590
591 public Listener(final String name) throws IOException {
592 super(name);
593
594 acceptChannel = ServerSocketChannel.open();
595 acceptChannel.configureBlocking(false);
596
597
598 bind(acceptChannel.socket(), isa, backlogLength);
599 port = acceptChannel.socket().getLocalPort();
600
601 selector= Selector.open();
602
603 readers = new Reader[readThreads];
604 readPool = Executors.newFixedThreadPool(readThreads,
605 new ThreadFactoryBuilder().setNameFormat(
606 "RpcServer.reader=%d,port=" + port).setDaemon(true).build());
607 for (int i = 0; i < readThreads; ++i) {
608 Reader reader = new Reader();
609 readers[i] = reader;
610 readPool.execute(reader);
611 }
612 LOG.info(getName() + ": started " + readThreads + " reader(s).");
613
614
615 acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
616 this.setName("RpcServer.listener,port=" + port);
617 this.setDaemon(true);
618 }
619
620
621 private class Reader implements Runnable {
622 private volatile boolean adding = false;
623 private final Selector readSelector;
624
625 Reader() throws IOException {
626 this.readSelector = Selector.open();
627 }
628 public void run() {
629 try {
630 doRunLoop();
631 } finally {
632 try {
633 readSelector.close();
634 } catch (IOException ioe) {
635 LOG.error(getName() + ": error closing read selector in " + getName(), ioe);
636 }
637 }
638 }
639
640 private synchronized void doRunLoop() {
641 while (running) {
642 SelectionKey key = null;
643 try {
644 readSelector.select();
645 while (adding) {
646 this.wait(1000);
647 }
648
649 Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
650 while (iter.hasNext()) {
651 key = iter.next();
652 iter.remove();
653 if (key.isValid()) {
654 if (key.isReadable()) {
655 doRead(key);
656 }
657 }
658 key = null;
659 }
660 } catch (InterruptedException e) {
661 if (running) {
662 LOG.info(getName() + ": unexpectedly interrupted: " +
663 StringUtils.stringifyException(e));
664 }
665 } catch (IOException ex) {
666 LOG.error(getName() + ": error in Reader", ex);
667 }
668 }
669 }
670
671
672
673
674
675
676
677
678 public void startAdd() {
679 adding = true;
680 readSelector.wakeup();
681 }
682
683 public synchronized SelectionKey registerChannel(SocketChannel channel)
684 throws IOException {
685 return channel.register(readSelector, SelectionKey.OP_READ);
686 }
687
688 public synchronized void finishAdd() {
689 adding = false;
690 this.notify();
691 }
692 }
693
694
695
696
697
698
699
700
701 private void cleanupConnections(boolean force) {
702 if (force || numConnections > thresholdIdleConnections) {
703 long currentTime = System.currentTimeMillis();
704 if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
705 return;
706 }
707 int start = 0;
708 int end = numConnections - 1;
709 if (!force) {
710 start = rand.nextInt() % numConnections;
711 end = rand.nextInt() % numConnections;
712 int temp;
713 if (end < start) {
714 temp = start;
715 start = end;
716 end = temp;
717 }
718 }
719 int i = start;
720 int numNuked = 0;
721 while (i <= end) {
722 Connection c;
723 synchronized (connectionList) {
724 try {
725 c = connectionList.get(i);
726 } catch (Exception e) {return;}
727 }
728 if (c.timedOut(currentTime)) {
729 if (LOG.isDebugEnabled())
730 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
731 closeConnection(c);
732 numNuked++;
733 end--;
734
735 c = null;
736 if (!force && numNuked == maxConnectionsToNuke) break;
737 }
738 else i++;
739 }
740 lastCleanupRunTime = System.currentTimeMillis();
741 }
742 }
743
744 @Override
745 public void run() {
746 LOG.info(getName() + ": starting");
747 while (running) {
748 SelectionKey key = null;
749 try {
750 selector.select();
751 Iterator<SelectionKey> iter = selector.selectedKeys().iterator();
752 while (iter.hasNext()) {
753 key = iter.next();
754 iter.remove();
755 try {
756 if (key.isValid()) {
757 if (key.isAcceptable())
758 doAccept(key);
759 }
760 } catch (IOException ignored) {
761 }
762 key = null;
763 }
764 } catch (OutOfMemoryError e) {
765 if (errorHandler != null) {
766 if (errorHandler.checkOOME(e)) {
767 LOG.info(getName() + ": exiting on OutOfMemoryError");
768 closeCurrentConnection(key, e);
769 cleanupConnections(true);
770 return;
771 }
772 } else {
773
774
775
776 LOG.warn(getName() + ": OutOfMemoryError in server select", e);
777 closeCurrentConnection(key, e);
778 cleanupConnections(true);
779 try { Thread.sleep(60000); } catch (Exception ignored) {}
780 }
781 } catch (Exception e) {
782 closeCurrentConnection(key, e);
783 }
784 cleanupConnections(false);
785 }
786 LOG.info(getName() + ": stopping");
787
788 synchronized (this) {
789 try {
790 acceptChannel.close();
791 selector.close();
792 } catch (IOException ignored) { }
793
794 selector= null;
795 acceptChannel= null;
796
797
798 while (!connectionList.isEmpty()) {
799 closeConnection(connectionList.remove(0));
800 }
801 }
802 }
803
804 private void closeCurrentConnection(SelectionKey key, Throwable e) {
805 if (key != null) {
806 Connection c = (Connection)key.attachment();
807 if (c != null) {
808 if (LOG.isDebugEnabled()) {
809 LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
810 (e != null ? " on error " + e.getMessage() : ""));
811 }
812 closeConnection(c);
813 key.attach(null);
814 }
815 }
816 }
817
818 InetSocketAddress getAddress() {
819 return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
820 }
821
822 void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
823 Connection c;
824 ServerSocketChannel server = (ServerSocketChannel) key.channel();
825
826 SocketChannel channel;
827 while ((channel = server.accept()) != null) {
828 try {
829 channel.configureBlocking(false);
830 channel.socket().setTcpNoDelay(tcpNoDelay);
831 channel.socket().setKeepAlive(tcpKeepAlive);
832 } catch (IOException ioe) {
833 channel.close();
834 throw ioe;
835 }
836
837 Reader reader = getReader();
838 try {
839 reader.startAdd();
840 SelectionKey readKey = reader.registerChannel(channel);
841 c = getConnection(channel, System.currentTimeMillis());
842 readKey.attach(c);
843 synchronized (connectionList) {
844 connectionList.add(numConnections, c);
845 numConnections++;
846 }
847 if (LOG.isDebugEnabled())
848 LOG.debug(getName() + ": connection from " + c.toString() +
849 "; # active connections: " + numConnections);
850 } finally {
851 reader.finishAdd();
852 }
853 }
854 }
855
856 void doRead(SelectionKey key) throws InterruptedException {
857 int count = 0;
858 Connection c = (Connection)key.attachment();
859 if (c == null) {
860 return;
861 }
862 c.setLastContact(System.currentTimeMillis());
863 try {
864 count = c.readAndProcess();
865 } catch (InterruptedException ieo) {
866 throw ieo;
867 } catch (Exception e) {
868 LOG.warn(getName() + ": count of bytes read: " + count, e);
869 count = -1;
870 }
871 if (count < 0) {
872 if (LOG.isDebugEnabled()) {
873 LOG.debug(getName() + ": DISCONNECTING client " + c.toString() +
874 " because read count=" + count +
875 ". Number of active connections: " + numConnections);
876 }
877 closeConnection(c);
878
879 } else {
880 c.setLastContact(System.currentTimeMillis());
881 }
882 }
883
884 synchronized void doStop() {
885 if (selector != null) {
886 selector.wakeup();
887 Thread.yield();
888 }
889 if (acceptChannel != null) {
890 try {
891 acceptChannel.socket().close();
892 } catch (IOException e) {
893 LOG.info(getName() + ": exception in closing listener socket. " + e);
894 }
895 }
896 readPool.shutdownNow();
897 }
898
899
900
901 Reader getReader() {
902 currentReader = (currentReader + 1) % readers.length;
903 return readers[currentReader];
904 }
905 }
906
907
908 protected class Responder extends Thread {
909 private final Selector writeSelector;
910 private int pending;
911
912 Responder() throws IOException {
913 this.setName("RpcServer.responder");
914 this.setDaemon(true);
915 writeSelector = Selector.open();
916 pending = 0;
917 }
918
919 @Override
920 public void run() {
921 LOG.info(getName() + ": starting");
922 try {
923 doRunLoop();
924 } finally {
925 LOG.info(getName() + ": stopping");
926 try {
927 writeSelector.close();
928 } catch (IOException ioe) {
929 LOG.error(getName() + ": couldn't close write selector", ioe);
930 }
931 }
932 }
933
934 private void doRunLoop() {
935 long lastPurgeTime = 0;
936
937 while (running) {
938 try {
939 waitPending();
940 writeSelector.select(purgeTimeout);
941 Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
942 while (iter.hasNext()) {
943 SelectionKey key = iter.next();
944 iter.remove();
945 try {
946 if (key.isValid() && key.isWritable()) {
947 doAsyncWrite(key);
948 }
949 } catch (IOException e) {
950 LOG.info(getName() + ": asyncWrite", e);
951 }
952 }
953 long now = System.currentTimeMillis();
954 if (now < lastPurgeTime + purgeTimeout) {
955 continue;
956 }
957 lastPurgeTime = now;
958
959
960
961
962 if (LOG.isDebugEnabled()) LOG.debug(getName() + ": checking for old call responses.");
963 ArrayList<Call> calls;
964
965
966 synchronized (writeSelector.keys()) {
967 calls = new ArrayList<Call>(writeSelector.keys().size());
968 iter = writeSelector.keys().iterator();
969 while (iter.hasNext()) {
970 SelectionKey key = iter.next();
971 Call call = (Call)key.attachment();
972 if (call != null && key.channel() == call.connection.channel) {
973 calls.add(call);
974 }
975 }
976 }
977
978 for(Call call : calls) {
979 try {
980 doPurge(call, now);
981 } catch (IOException e) {
982 LOG.warn(getName() + ": error in purging old calls " + e);
983 }
984 }
985 } catch (OutOfMemoryError e) {
986 if (errorHandler != null) {
987 if (errorHandler.checkOOME(e)) {
988 LOG.info(getName() + ": exiting on OutOfMemoryError");
989 return;
990 }
991 } else {
992
993
994
995
996
997 LOG.warn(getName() + ": OutOfMemoryError in server select", e);
998 try { Thread.sleep(60000); } catch (Exception ignored) {}
999 }
1000 } catch (Exception e) {
1001 LOG.warn(getName() + ": exception in Responder " +
1002 StringUtils.stringifyException(e));
1003 }
1004 }
1005 LOG.info(getName() + ": stopped");
1006 }
1007
1008 private void doAsyncWrite(SelectionKey key) throws IOException {
1009 Call call = (Call)key.attachment();
1010 if (call == null) {
1011 return;
1012 }
1013 if (key.channel() != call.connection.channel) {
1014 throw new IOException("doAsyncWrite: bad channel");
1015 }
1016
1017 synchronized(call.connection.responseQueue) {
1018 if (processResponse(call.connection.responseQueue, false)) {
1019 try {
1020 key.interestOps(0);
1021 } catch (CancelledKeyException e) {
1022
1023
1024
1025
1026
1027 LOG.warn("Exception while changing ops : " + e);
1028 }
1029 }
1030 }
1031 }
1032
1033
1034
1035
1036
1037 private void doPurge(Call call, long now) throws IOException {
1038 synchronized (call.connection.responseQueue) {
1039 Iterator<Call> iter = call.connection.responseQueue.listIterator(0);
1040 while (iter.hasNext()) {
1041 Call nextCall = iter.next();
1042 if (now > nextCall.timestamp + purgeTimeout) {
1043 closeConnection(nextCall.connection);
1044 break;
1045 }
1046 }
1047 }
1048 }
1049
1050
1051
1052
1053 private boolean processResponse(final LinkedList<Call> responseQueue, boolean inHandler)
1054 throws IOException {
1055 boolean error = true;
1056 boolean done = false;
1057 int numElements;
1058 Call call = null;
1059 try {
1060
1061 synchronized (responseQueue) {
1062
1063
1064
1065 numElements = responseQueue.size();
1066 if (numElements == 0) {
1067 error = false;
1068 return true;
1069 }
1070
1071
1072
1073 call = responseQueue.removeFirst();
1074 SocketChannel channel = call.connection.channel;
1075
1076
1077
1078 long numBytes = channelWrite(channel, call.response);
1079 if (numBytes < 0) {
1080 return true;
1081 }
1082 if (!call.response.hasRemaining()) {
1083 call.connection.decRpcCount();
1084
1085 if (numElements == 1) {
1086 done = true;
1087 } else {
1088 done = false;
1089 }
1090 if (LOG.isDebugEnabled()) {
1091 LOG.debug(getName() + ": callId: " + call.id + " wrote " + numBytes + " bytes.");
1092 }
1093 } else {
1094
1095
1096
1097
1098 call.connection.responseQueue.addFirst(call);
1099
1100 if (inHandler) {
1101
1102 call.timestamp = System.currentTimeMillis();
1103 if (enqueueInSelector(call))
1104 done = true;
1105 }
1106 if (LOG.isDebugEnabled()) {
1107 LOG.debug(getName() + call.toShortString() + " partially sent, wrote " +
1108 numBytes + " bytes.");
1109 }
1110 }
1111 error = false;
1112 }
1113 } finally {
1114 if (error && call != null) {
1115 LOG.warn(getName() + call.toShortString() + ": output error");
1116 done = true;
1117 closeConnection(call.connection);
1118 }
1119 }
1120 if (done) call.done();
1121 return done;
1122 }
1123
1124
1125
1126
1127 private boolean enqueueInSelector(Call call) throws IOException {
1128 boolean done = false;
1129 incPending();
1130 try {
1131
1132
1133 SocketChannel channel = call.connection.channel;
1134 writeSelector.wakeup();
1135 channel.register(writeSelector, SelectionKey.OP_WRITE, call);
1136 } catch (ClosedChannelException e) {
1137
1138 done = true;
1139 } finally {
1140 decPending();
1141 }
1142 return done;
1143 }
1144
1145
1146
1147
1148 void doRespond(Call call) throws IOException {
1149
1150 call.timestamp = System.currentTimeMillis();
1151
1152 boolean doRegister = false;
1153 synchronized (call.connection.responseQueue) {
1154 call.connection.responseQueue.addLast(call);
1155 if (call.connection.responseQueue.size() == 1) {
1156 doRegister = !processResponse(call.connection.responseQueue, false);
1157 }
1158 }
1159 if (doRegister) {
1160 enqueueInSelector(call);
1161 }
1162 }
1163
1164 private synchronized void incPending() {
1165 pending++;
1166 }
1167
1168 private synchronized void decPending() {
1169 pending--;
1170 notify();
1171 }
1172
1173 private synchronized void waitPending() throws InterruptedException {
1174 while (pending > 0) {
1175 wait();
1176 }
1177 }
1178 }
1179
1180 @SuppressWarnings("serial")
1181 public static class CallQueueTooBigException extends IOException {
1182 CallQueueTooBigException() {
1183 super();
1184 }
1185 }
1186
1187
1188 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
1189 value="VO_VOLATILE_INCREMENT",
1190 justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")
1191 public class Connection {
1192
1193 private boolean connectionPreambleRead = false;
1194
1195 private boolean connectionHeaderRead = false;
1196 protected SocketChannel channel;
1197 private ByteBuffer data;
1198 private ByteBuffer dataLengthBuffer;
1199 protected final LinkedList<Call> responseQueue;
1200 private Counter rpcCount = new Counter();
1201 private long lastContact;
1202 private InetAddress addr;
1203 protected Socket socket;
1204
1205
1206 protected String hostAddress;
1207 protected int remotePort;
1208 ConnectionHeader connectionHeader;
1209
1210
1211
1212 private Codec codec;
1213
1214
1215
1216 private CompressionCodec compressionCodec;
1217 BlockingService service;
1218 protected UserGroupInformation user = null;
1219 private AuthMethod authMethod;
1220 private boolean saslContextEstablished;
1221 private boolean skipInitialSaslHandshake;
1222 private ByteBuffer unwrappedData;
1223
1224 private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
1225 boolean useSasl;
1226 SaslServer saslServer;
1227 private boolean useWrap = false;
1228
1229 private static final int AUTHROIZATION_FAILED_CALLID = -1;
1230 private final Call authFailedCall =
1231 new Call(AUTHROIZATION_FAILED_CALLID, this.service, null,
1232 null, null, null, this, null, 0, null, null);
1233 private ByteArrayOutputStream authFailedResponse =
1234 new ByteArrayOutputStream();
1235
1236 private static final int SASL_CALLID = -33;
1237 private final Call saslCall =
1238 new Call(SASL_CALLID, this.service, null, null, null, null, this, null, 0, null, null);
1239
1240
1241 private boolean authenticatedWithFallback;
1242
1243 public UserGroupInformation attemptingUser = null;
1244
1245 public Connection(SocketChannel channel, long lastContact) {
1246 this.channel = channel;
1247 this.lastContact = lastContact;
1248 this.data = null;
1249 this.dataLengthBuffer = ByteBuffer.allocate(4);
1250 this.socket = channel.socket();
1251 this.addr = socket.getInetAddress();
1252 if (addr == null) {
1253 this.hostAddress = "*Unknown*";
1254 } else {
1255 this.hostAddress = addr.getHostAddress();
1256 }
1257 this.remotePort = socket.getPort();
1258 this.responseQueue = new LinkedList<Call>();
1259 if (socketSendBufferSize != 0) {
1260 try {
1261 socket.setSendBufferSize(socketSendBufferSize);
1262 } catch (IOException e) {
1263 LOG.warn("Connection: unable to set socket send buffer size to " +
1264 socketSendBufferSize);
1265 }
1266 }
1267 }
1268
1269 @Override
1270 public String toString() {
1271 return getHostAddress() + ":" + remotePort;
1272 }
1273
1274 public String getHostAddress() {
1275 return hostAddress;
1276 }
1277
1278 public InetAddress getHostInetAddress() {
1279 return addr;
1280 }
1281
1282 public int getRemotePort() {
1283 return remotePort;
1284 }
1285
1286 public void setLastContact(long lastContact) {
1287 this.lastContact = lastContact;
1288 }
1289
1290 public long getLastContact() {
1291 return lastContact;
1292 }
1293
1294
1295 private boolean isIdle() {
1296 return rpcCount.get() == 0;
1297 }
1298
1299
1300 protected void decRpcCount() {
1301 rpcCount.decrement();
1302 }
1303
1304
1305 protected void incRpcCount() {
1306 rpcCount.increment();
1307 }
1308
1309 protected boolean timedOut(long currentTime) {
1310 return isIdle() && currentTime - lastContact > maxIdleTime;
1311 }
1312
1313 private UserGroupInformation getAuthorizedUgi(String authorizedId)
1314 throws IOException {
1315 UserGroupInformation authorizedUgi;
1316 if (authMethod == AuthMethod.DIGEST) {
1317 TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
1318 secretManager);
1319 authorizedUgi = tokenId.getUser();
1320 if (authorizedUgi == null) {
1321 throw new AccessDeniedException(
1322 "Can't retrieve username from tokenIdentifier.");
1323 }
1324 authorizedUgi.addTokenIdentifier(tokenId);
1325 } else {
1326 authorizedUgi = UserGroupInformation.createRemoteUser(authorizedId);
1327 }
1328 authorizedUgi.setAuthenticationMethod(authMethod.authenticationMethod);
1329 return authorizedUgi;
1330 }
1331
1332 private void saslReadAndProcess(byte[] saslToken) throws IOException,
1333 InterruptedException {
1334 if (saslContextEstablished) {
1335 if (LOG.isDebugEnabled())
1336 LOG.debug("Have read input token of size " + saslToken.length
1337 + " for processing by saslServer.unwrap()");
1338
1339 if (!useWrap) {
1340 processOneRpc(saslToken);
1341 } else {
1342 byte [] plaintextData = saslServer.unwrap(saslToken, 0, saslToken.length);
1343 processUnwrappedData(plaintextData);
1344 }
1345 } else {
1346 byte[] replyToken = null;
1347 try {
1348 if (saslServer == null) {
1349 switch (authMethod) {
1350 case DIGEST:
1351 if (secretManager == null) {
1352 throw new AccessDeniedException(
1353 "Server is not configured to do DIGEST authentication.");
1354 }
1355 saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
1356 .getMechanismName(), null, SaslUtil.SASL_DEFAULT_REALM,
1357 SaslUtil.SASL_PROPS, new SaslDigestCallbackHandler(
1358 secretManager, this));
1359 break;
1360 default:
1361 UserGroupInformation current = UserGroupInformation
1362 .getCurrentUser();
1363 String fullName = current.getUserName();
1364 if (LOG.isDebugEnabled()) {
1365 LOG.debug("Kerberos principal name is " + fullName);
1366 }
1367 final String names[] = SaslUtil.splitKerberosName(fullName);
1368 if (names.length != 3) {
1369 throw new AccessDeniedException(
1370 "Kerberos principal name does NOT have the expected "
1371 + "hostname part: " + fullName);
1372 }
1373 current.doAs(new PrivilegedExceptionAction<Object>() {
1374 @Override
1375 public Object run() throws SaslException {
1376 saslServer = Sasl.createSaslServer(AuthMethod.KERBEROS
1377 .getMechanismName(), names[0], names[1],
1378 SaslUtil.SASL_PROPS, new SaslGssCallbackHandler());
1379 return null;
1380 }
1381 });
1382 }
1383 if (saslServer == null)
1384 throw new AccessDeniedException(
1385 "Unable to find SASL server implementation for "
1386 + authMethod.getMechanismName());
1387 if (LOG.isDebugEnabled()) {
1388 LOG.debug("Created SASL server with mechanism = " + authMethod.getMechanismName());
1389 }
1390 }
1391 if (LOG.isDebugEnabled()) {
1392 LOG.debug("Have read input token of size " + saslToken.length
1393 + " for processing by saslServer.evaluateResponse()");
1394 }
1395 replyToken = saslServer.evaluateResponse(saslToken);
1396 } catch (IOException e) {
1397 IOException sendToClient = e;
1398 Throwable cause = e;
1399 while (cause != null) {
1400 if (cause instanceof InvalidToken) {
1401 sendToClient = (InvalidToken) cause;
1402 break;
1403 }
1404 cause = cause.getCause();
1405 }
1406 doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(),
1407 sendToClient.getLocalizedMessage());
1408 metrics.authenticationFailure();
1409 String clientIP = this.toString();
1410
1411 AUDITLOG.warn(AUTH_FAILED_FOR + clientIP + ":" + attemptingUser);
1412 throw e;
1413 }
1414 if (replyToken != null) {
1415 if (LOG.isDebugEnabled()) {
1416 LOG.debug("Will send token of size " + replyToken.length
1417 + " from saslServer.");
1418 }
1419 doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null,
1420 null);
1421 }
1422 if (saslServer.isComplete()) {
1423 String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
1424 useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
1425 user = getAuthorizedUgi(saslServer.getAuthorizationID());
1426 if (LOG.isDebugEnabled()) {
1427 LOG.debug("SASL server context established. Authenticated client: "
1428 + user + ". Negotiated QoP is "
1429 + saslServer.getNegotiatedProperty(Sasl.QOP));
1430 }
1431 metrics.authenticationSuccess();
1432 AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user);
1433 saslContextEstablished = true;
1434 }
1435 }
1436 }
1437
1438
1439
1440
1441 private void doRawSaslReply(SaslStatus status, Writable rv,
1442 String errorClass, String error) throws IOException {
1443 ByteBufferOutputStream saslResponse = null;
1444 DataOutputStream out = null;
1445 try {
1446
1447
1448 saslResponse = new ByteBufferOutputStream(256);
1449 out = new DataOutputStream(saslResponse);
1450 out.writeInt(status.state);
1451 if (status == SaslStatus.SUCCESS) {
1452 rv.write(out);
1453 } else {
1454 WritableUtils.writeString(out, errorClass);
1455 WritableUtils.writeString(out, error);
1456 }
1457 saslCall.setSaslTokenResponse(saslResponse.getByteBuffer());
1458 saslCall.responder = responder;
1459 saslCall.sendResponseIfReady();
1460 } finally {
1461 if (saslResponse != null) {
1462 saslResponse.close();
1463 }
1464 if (out != null) {
1465 out.close();
1466 }
1467 }
1468 }
1469
1470 private void disposeSasl() {
1471 if (saslServer != null) {
1472 try {
1473 saslServer.dispose();
1474 saslServer = null;
1475 } catch (SaslException ignored) {
1476
1477 }
1478 }
1479 }
1480
1481
1482
1483
1484
1485
1486
1487
1488 public int readAndProcess() throws IOException, InterruptedException {
1489 while (true) {
1490
1491
1492
1493
1494 int count;
1495 if (this.dataLengthBuffer.remaining() > 0) {
1496 count = channelRead(channel, this.dataLengthBuffer);
1497 if (count < 0 || this.dataLengthBuffer.remaining() > 0) {
1498 return count;
1499 }
1500 }
1501
1502 if (!connectionPreambleRead) {
1503
1504 this.dataLengthBuffer.flip();
1505 if (!HConstants.RPC_HEADER.equals(dataLengthBuffer)) {
1506 return doBadPreambleHandling("Expected HEADER=" +
1507 Bytes.toStringBinary(HConstants.RPC_HEADER.array()) +
1508 " but received HEADER=" + Bytes.toStringBinary(dataLengthBuffer.array()) +
1509 " from " + toString());
1510 }
1511
1512 ByteBuffer versionAndAuthBytes = ByteBuffer.allocate(2);
1513 count = channelRead(channel, versionAndAuthBytes);
1514 if (count < 0 || versionAndAuthBytes.remaining() > 0) {
1515 return count;
1516 }
1517 int version = versionAndAuthBytes.get(0);
1518 byte authbyte = versionAndAuthBytes.get(1);
1519 this.authMethod = AuthMethod.valueOf(authbyte);
1520 if (version != CURRENT_VERSION) {
1521 String msg = getFatalConnectionString(version, authbyte);
1522 return doBadPreambleHandling(msg, new WrongVersionException(msg));
1523 }
1524 if (authMethod == null) {
1525 String msg = getFatalConnectionString(version, authbyte);
1526 return doBadPreambleHandling(msg, new BadAuthException(msg));
1527 }
1528 if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
1529 if (allowFallbackToSimpleAuth) {
1530 metrics.authenticationFallback();
1531 authenticatedWithFallback = true;
1532 } else {
1533 AccessDeniedException ae = new AccessDeniedException("Authentication is required");
1534 setupResponse(authFailedResponse, authFailedCall, ae, ae.getMessage());
1535 responder.doRespond(authFailedCall);
1536 throw ae;
1537 }
1538 }
1539 if (!isSecurityEnabled && authMethod != AuthMethod.SIMPLE) {
1540 doRawSaslReply(SaslStatus.SUCCESS, new IntWritable(
1541 SaslUtil.SWITCH_TO_SIMPLE_AUTH), null, null);
1542 authMethod = AuthMethod.SIMPLE;
1543
1544
1545
1546 skipInitialSaslHandshake = true;
1547 }
1548 if (authMethod != AuthMethod.SIMPLE) {
1549 useSasl = true;
1550 }
1551 connectionPreambleRead = true;
1552
1553 dataLengthBuffer.clear();
1554 continue;
1555 }
1556
1557
1558 if (data == null) {
1559 dataLengthBuffer.flip();
1560 int dataLength = dataLengthBuffer.getInt();
1561 if (dataLength == RpcClient.PING_CALL_ID) {
1562 if (!useWrap) {
1563 dataLengthBuffer.clear();
1564 return 0;
1565 }
1566 }
1567 if (dataLength < 0) {
1568 throw new IllegalArgumentException("Unexpected data length "
1569 + dataLength + "!! from " + getHostAddress());
1570 }
1571 data = ByteBuffer.allocate(dataLength);
1572 incRpcCount();
1573 }
1574 count = channelRead(channel, data);
1575 if (count < 0) {
1576 return count;
1577 } else if (data.remaining() == 0) {
1578 dataLengthBuffer.clear();
1579 data.flip();
1580 if (skipInitialSaslHandshake) {
1581 data = null;
1582 skipInitialSaslHandshake = false;
1583 continue;
1584 }
1585 boolean headerRead = connectionHeaderRead;
1586 if (useSasl) {
1587 saslReadAndProcess(data.array());
1588 } else {
1589 processOneRpc(data.array());
1590 }
1591 this.data = null;
1592 if (!headerRead) {
1593 continue;
1594 }
1595 } else if (count > 0) {
1596
1597 if (LOG.isTraceEnabled()) LOG.trace("Continue to read rest of data " + data.remaining());
1598 continue;
1599 }
1600 return count;
1601 }
1602 }
1603
1604 private String getFatalConnectionString(final int version, final byte authByte) {
1605 return "serverVersion=" + CURRENT_VERSION +
1606 ", clientVersion=" + version + ", authMethod=" + authByte +
1607 ", authSupported=" + (authMethod != null) + " from " + toString();
1608 }
1609
1610 private int doBadPreambleHandling(final String msg) throws IOException {
1611 return doBadPreambleHandling(msg, new FatalConnectionException(msg));
1612 }
1613
1614 private int doBadPreambleHandling(final String msg, final Exception e) throws IOException {
1615 LOG.warn(msg);
1616 Call fakeCall = new Call(-1, null, null, null, null, null, this, responder, -1, null, null);
1617 setupResponse(null, fakeCall, e, msg);
1618 responder.doRespond(fakeCall);
1619
1620 return -1;
1621 }
1622
1623
1624 private void processConnectionHeader(byte[] buf) throws IOException {
1625 this.connectionHeader = ConnectionHeader.parseFrom(buf);
1626 String serviceName = connectionHeader.getServiceName();
1627 if (serviceName == null) throw new EmptyServiceNameException();
1628 this.service = getService(services, serviceName);
1629 if (this.service == null) throw new UnknownServiceException(serviceName);
1630 setupCellBlockCodecs(this.connectionHeader);
1631 UserGroupInformation protocolUser = createUser(connectionHeader);
1632 if (!useSasl) {
1633 user = protocolUser;
1634 if (user != null) {
1635 user.setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
1636 }
1637
1638 if (authenticatedWithFallback) {
1639 LOG.warn("Allowed fallback to SIMPLE auth for " + user
1640 + " connecting from " + getHostAddress());
1641 }
1642 AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user);
1643 } else {
1644
1645 user.setAuthenticationMethod(authMethod.authenticationMethod);
1646
1647
1648
1649 if ((protocolUser != null)
1650 && (!protocolUser.getUserName().equals(user.getUserName()))) {
1651 if (authMethod == AuthMethod.DIGEST) {
1652
1653 throw new AccessDeniedException("Authenticated user (" + user
1654 + ") doesn't match what the client claims to be ("
1655 + protocolUser + ")");
1656 } else {
1657
1658
1659
1660 UserGroupInformation realUser = user;
1661 user = UserGroupInformation.createProxyUser(protocolUser
1662 .getUserName(), realUser);
1663
1664 user.setAuthenticationMethod(AuthenticationMethod.PROXY);
1665 }
1666 }
1667 }
1668 if (connectionHeader.hasVersionInfo()) {
1669 AUDITLOG.info("Connection from " + this.hostAddress + " port: " + this.remotePort
1670 + " with version info: "
1671 + TextFormat.shortDebugString(connectionHeader.getVersionInfo()));
1672 } else {
1673 AUDITLOG.info("Connection from " + this.hostAddress + " port: " + this.remotePort
1674 + " with unknown version info");
1675 }
1676 }
1677
1678
1679
1680
1681
1682
1683 private void setupCellBlockCodecs(final ConnectionHeader header)
1684 throws FatalConnectionException {
1685
1686 if (!header.hasCellBlockCodecClass()) return;
1687 String className = header.getCellBlockCodecClass();
1688 if (className == null || className.length() == 0) return;
1689 try {
1690 this.codec = (Codec)Class.forName(className).newInstance();
1691 } catch (Exception e) {
1692 throw new UnsupportedCellCodecException(className, e);
1693 }
1694 if (!header.hasCellBlockCompressorClass()) return;
1695 className = header.getCellBlockCompressorClass();
1696 try {
1697 this.compressionCodec = (CompressionCodec)Class.forName(className).newInstance();
1698 } catch (Exception e) {
1699 throw new UnsupportedCompressionCodecException(className, e);
1700 }
1701 }
1702
1703 private void processUnwrappedData(byte[] inBuf) throws IOException,
1704 InterruptedException {
1705 ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf));
1706
1707 while (true) {
1708 int count = -1;
1709 if (unwrappedDataLengthBuffer.remaining() > 0) {
1710 count = channelRead(ch, unwrappedDataLengthBuffer);
1711 if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0)
1712 return;
1713 }
1714
1715 if (unwrappedData == null) {
1716 unwrappedDataLengthBuffer.flip();
1717 int unwrappedDataLength = unwrappedDataLengthBuffer.getInt();
1718
1719 if (unwrappedDataLength == RpcClient.PING_CALL_ID) {
1720 if (LOG.isDebugEnabled())
1721 LOG.debug("Received ping message");
1722 unwrappedDataLengthBuffer.clear();
1723 continue;
1724 }
1725 unwrappedData = ByteBuffer.allocate(unwrappedDataLength);
1726 }
1727
1728 count = channelRead(ch, unwrappedData);
1729 if (count <= 0 || unwrappedData.remaining() > 0)
1730 return;
1731
1732 if (unwrappedData.remaining() == 0) {
1733 unwrappedDataLengthBuffer.clear();
1734 unwrappedData.flip();
1735 processOneRpc(unwrappedData.array());
1736 unwrappedData = null;
1737 }
1738 }
1739 }
1740
1741 private void processOneRpc(byte[] buf) throws IOException, InterruptedException {
1742 if (connectionHeaderRead) {
1743 processRequest(buf);
1744 } else {
1745 processConnectionHeader(buf);
1746 this.connectionHeaderRead = true;
1747 if (!authorizeConnection()) {
1748
1749
1750 throw new AccessDeniedException("Connection from " + this + " for service " +
1751 connectionHeader.getServiceName() + " is unauthorized for user: " + user);
1752 }
1753 }
1754 }
1755
1756
1757
1758
1759
1760
1761
1762 protected void processRequest(byte[] buf) throws IOException, InterruptedException {
1763 long totalRequestSize = buf.length;
1764 int offset = 0;
1765
1766
1767 CodedInputStream cis = CodedInputStream.newInstance(buf, offset, buf.length);
1768 int headerSize = cis.readRawVarint32();
1769 offset = cis.getTotalBytesRead();
1770 Message.Builder builder = RequestHeader.newBuilder();
1771 ProtobufUtil.mergeFrom(builder, buf, offset, headerSize);
1772 RequestHeader header = (RequestHeader) builder.build();
1773 offset += headerSize;
1774 int id = header.getCallId();
1775 if (LOG.isTraceEnabled()) {
1776 LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) +
1777 " totalRequestSize: " + totalRequestSize + " bytes");
1778 }
1779
1780
1781 if ((totalRequestSize + callQueueSize.get()) > maxQueueSize) {
1782 final Call callTooBig =
1783 new Call(id, this.service, null, null, null, null, this,
1784 responder, totalRequestSize, null, null);
1785 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1786 metrics.exception(CALL_QUEUE_TOO_BIG_EXCEPTION);
1787 InetSocketAddress address = getListenerAddress();
1788 setupResponse(responseBuffer, callTooBig, CALL_QUEUE_TOO_BIG_EXCEPTION,
1789 "Call queue is full on " + (address != null ? address : "(channel closed)") +
1790 ", is hbase.ipc.server.max.callqueue.size too small?");
1791 responder.doRespond(callTooBig);
1792 return;
1793 }
1794 MethodDescriptor md = null;
1795 Message param = null;
1796 CellScanner cellScanner = null;
1797 try {
1798 if (header.hasRequestParam() && header.getRequestParam()) {
1799 md = this.service.getDescriptorForType().findMethodByName(header.getMethodName());
1800 if (md == null) throw new UnsupportedOperationException(header.getMethodName());
1801 builder = this.service.getRequestPrototype(md).newBuilderForType();
1802
1803 cis = CodedInputStream.newInstance(buf, offset, buf.length);
1804 int paramSize = cis.readRawVarint32();
1805 offset += cis.getTotalBytesRead();
1806 if (builder != null) {
1807 ProtobufUtil.mergeFrom(builder, buf, offset, paramSize);
1808 param = builder.build();
1809 }
1810 offset += paramSize;
1811 }
1812 if (header.hasCellBlockMeta()) {
1813 cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec,
1814 buf, offset, buf.length);
1815 }
1816 } catch (Throwable t) {
1817 InetSocketAddress address = getListenerAddress();
1818 String msg = (address != null ? address : "(channel closed)") +
1819 " is unable to read call parameter from client " + getHostAddress();
1820 LOG.warn(msg, t);
1821
1822 metrics.exception(t);
1823
1824
1825 if (t instanceof LinkageError) {
1826 t = new DoNotRetryIOException(t);
1827 }
1828
1829 if (t instanceof UnsupportedOperationException) {
1830 t = new DoNotRetryIOException(t);
1831 }
1832
1833 final Call readParamsFailedCall =
1834 new Call(id, this.service, null, null, null, null, this,
1835 responder, totalRequestSize, null, null);
1836 ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
1837 setupResponse(responseBuffer, readParamsFailedCall, t,
1838 msg + "; " + t.getMessage());
1839 responder.doRespond(readParamsFailedCall);
1840 return;
1841 }
1842
1843 TraceInfo traceInfo = header.hasTraceInfo()
1844 ? new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId())
1845 : null;
1846 Call call = new Call(id, this.service, md, header, param, cellScanner, this, responder,
1847 totalRequestSize,
1848 traceInfo, RpcServer.getRemoteIp());
1849 scheduler.dispatch(new CallRunner(RpcServer.this, call));
1850 }
1851
1852 private boolean authorizeConnection() throws IOException {
1853 try {
1854
1855
1856
1857
1858 if (user != null && user.getRealUser() != null
1859 && (authMethod != AuthMethod.DIGEST)) {
1860 ProxyUsers.authorize(user, this.getHostAddress(), conf);
1861 }
1862 authorize(user, connectionHeader, getHostInetAddress());
1863 if (LOG.isDebugEnabled()) {
1864 LOG.debug("Authorized " + TextFormat.shortDebugString(connectionHeader));
1865 }
1866 metrics.authorizationSuccess();
1867 } catch (AuthorizationException ae) {
1868 LOG.debug("Connection authorization failed: " + ae.getMessage(), ae);
1869 metrics.authorizationFailure();
1870 setupResponse(authFailedResponse, authFailedCall,
1871 new AccessDeniedException(ae), ae.getMessage());
1872 responder.doRespond(authFailedCall);
1873 return false;
1874 }
1875 return true;
1876 }
1877
1878 protected synchronized void close() {
1879 disposeSasl();
1880 data = null;
1881 if (!channel.isOpen())
1882 return;
1883 try {socket.shutdownOutput();} catch(Exception ignored) {}
1884 if (channel.isOpen()) {
1885 try {channel.close();} catch(Exception ignored) {}
1886 }
1887 try {socket.close();} catch(Exception ignored) {}
1888 }
1889
1890 private UserGroupInformation createUser(ConnectionHeader head) {
1891 UserGroupInformation ugi = null;
1892
1893 if (!head.hasUserInfo()) {
1894 return null;
1895 }
1896 UserInformation userInfoProto = head.getUserInfo();
1897 String effectiveUser = null;
1898 if (userInfoProto.hasEffectiveUser()) {
1899 effectiveUser = userInfoProto.getEffectiveUser();
1900 }
1901 String realUser = null;
1902 if (userInfoProto.hasRealUser()) {
1903 realUser = userInfoProto.getRealUser();
1904 }
1905 if (effectiveUser != null) {
1906 if (realUser != null) {
1907 UserGroupInformation realUserUgi =
1908 UserGroupInformation.createRemoteUser(realUser);
1909 ugi = UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
1910 } else {
1911 ugi = UserGroupInformation.createRemoteUser(effectiveUser);
1912 }
1913 }
1914 return ugi;
1915 }
1916 }
1917
1918
1919
1920
1921
1922
1923
1924 public static class BlockingServiceAndInterface {
1925 private final BlockingService service;
1926 private final Class<?> serviceInterface;
1927 public BlockingServiceAndInterface(final BlockingService service,
1928 final Class<?> serviceInterface) {
1929 this.service = service;
1930 this.serviceInterface = serviceInterface;
1931 }
1932 public Class<?> getServiceInterface() {
1933 return this.serviceInterface;
1934 }
1935 public BlockingService getBlockingService() {
1936 return this.service;
1937 }
1938 }
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950 public RpcServer(final Server serverInstance, final String name,
1951 final List<BlockingServiceAndInterface> services,
1952 final InetSocketAddress isa, Configuration conf,
1953 RpcScheduler scheduler)
1954 throws IOException {
1955 this.serverInstance = serverInstance;
1956 this.reservoir = new BoundedByteBufferPool(
1957 conf.getInt("hbase.ipc.server.reservoir.max.buffer.size", 1024 * 1024),
1958 conf.getInt("hbase.ipc.server.reservoir.initial.buffer.size", 16 * 1024),
1959
1960 conf.getInt("hbase.ipc.server.reservoir.initial.max",
1961 conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
1962 HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT) * 2));
1963 this.services = services;
1964 this.isa = isa;
1965 this.conf = conf;
1966 this.socketSendBufferSize = 0;
1967 this.maxQueueSize = conf.getInt("hbase.ipc.server.max.callqueue.size",
1968 conf.getInt("ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE));
1969 this.readThreads = conf.getInt("hbase.ipc.server.read.threadpool.size",
1970 conf.getInt("ipc.server.read.threadpool.size", 10));
1971 this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime",
1972 conf.getInt("ipc.client.connection.maxidletime", 1000));
1973 this.maxConnectionsToNuke = conf.getInt("hbase.ipc.client.kill.max",
1974 conf.getInt("ipc.client.kill.max", 10));
1975 this.thresholdIdleConnections = conf.getInt("hbase.ipc.client.idlethreshold",
1976 conf.getInt("ipc.client.idlethreshold", 4000));
1977 this.purgeTimeout = conf.getLong("hbase.ipc.client.call.purge.timeout",
1978 conf.getLong("ipc.client.call.purge.timeout",
1979 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
1980 this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME);
1981 this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE);
1982
1983
1984 listener = new Listener(name);
1985 this.port = listener.getAddress().getPort();
1986
1987 this.metrics = new MetricsHBaseServer(name, new MetricsHBaseServerWrapperImpl(this));
1988 this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true);
1989 this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive",
1990 conf.getBoolean("ipc.server.tcpkeepalive", true));
1991
1992 this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS, DEFAULT_WARN_DELAYED_CALLS);
1993 this.delayedCalls = new AtomicInteger(0);
1994 this.ipcUtil = new IPCUtil(conf);
1995
1996
1997
1998 responder = new Responder();
1999 this.authorize = conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
2000 this.userProvider = UserProvider.instantiate(conf);
2001 this.isSecurityEnabled = userProvider.isHBaseSecurityEnabled();
2002 if (isSecurityEnabled) {
2003 HBaseSaslRpcServer.init(conf);
2004 }
2005 this.allowFallbackToSimpleAuth = conf.getBoolean(FALLBACK_TO_INSECURE_CLIENT_AUTH, false);
2006 if (isSecurityEnabled && allowFallbackToSimpleAuth) {
2007 LOG.warn("********* WARNING! *********");
2008 LOG.warn("This server is configured to allow connections from INSECURE clients");
2009 LOG.warn("(" + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = true).");
2010 LOG.warn("While this option is enabled, client identities cannot be secured, and user");
2011 LOG.warn("impersonation is possible!");
2012 LOG.warn("For secure operation, please disable SIMPLE authentication as soon as possible,");
2013 LOG.warn("by setting " + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = false in hbase-site.xml");
2014 LOG.warn("****************************");
2015 }
2016
2017 this.scheduler = scheduler;
2018 this.scheduler.init(new RpcSchedulerContext(this));
2019 }
2020
2021
2022
2023
2024
2025 protected Connection getConnection(SocketChannel channel, long time) {
2026 return new Connection(channel, time);
2027 }
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038 private void setupResponse(ByteArrayOutputStream response, Call call, Throwable t, String error)
2039 throws IOException {
2040 if (response != null) response.reset();
2041 call.setResponse(null, null, t, error);
2042 }
2043
2044 protected void closeConnection(Connection connection) {
2045 synchronized (connectionList) {
2046 if (connectionList.remove(connection)) {
2047 numConnections--;
2048 }
2049 }
2050 connection.close();
2051 }
2052
2053 Configuration getConf() {
2054 return conf;
2055 }
2056
2057
2058
2059
2060 @Override
2061 public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; }
2062
2063
2064 @Override
2065 public void start() {
2066 startThreads();
2067 openServer();
2068 }
2069
2070
2071
2072
2073 @Override
2074 public void openServer() {
2075 this.started = true;
2076 }
2077
2078 @Override
2079 public boolean isStarted() {
2080 return this.started;
2081 }
2082
2083
2084
2085
2086
2087 @Override
2088 public synchronized void startThreads() {
2089 authTokenSecretMgr = createSecretManager();
2090 if (authTokenSecretMgr != null) {
2091 setSecretManager(authTokenSecretMgr);
2092 authTokenSecretMgr.start();
2093 }
2094 this.authManager = new ServiceAuthorizationManager();
2095 HBasePolicyProvider.init(conf, authManager);
2096 responder.start();
2097 listener.start();
2098 scheduler.start();
2099 }
2100
2101 @Override
2102 public void refreshAuthManager(PolicyProvider pp) {
2103
2104
2105 this.authManager.refresh(this.conf, pp);
2106 }
2107
2108 private AuthenticationTokenSecretManager createSecretManager() {
2109 if (!isSecurityEnabled) return null;
2110 if (serverInstance == null) return null;
2111 if (!(serverInstance instanceof org.apache.hadoop.hbase.Server)) return null;
2112 org.apache.hadoop.hbase.Server server = (org.apache.hadoop.hbase.Server)serverInstance;
2113 Configuration conf = server.getConfiguration();
2114 long keyUpdateInterval =
2115 conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000);
2116 long maxAge =
2117 conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000);
2118 return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(),
2119 server.getServerName().toString(), keyUpdateInterval, maxAge);
2120 }
2121
2122 public SecretManager<? extends TokenIdentifier> getSecretManager() {
2123 return this.secretManager;
2124 }
2125
2126 @SuppressWarnings("unchecked")
2127 public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) {
2128 this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
2129 }
2130
2131
2132
2133
2134
2135
2136 public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
2137 Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
2138 throws IOException {
2139 try {
2140 status.setRPC(md.getName(), new Object[]{param}, receiveTime);
2141
2142 status.setRPCPacket(param);
2143 status.resume("Servicing call");
2144
2145 long startTime = System.currentTimeMillis();
2146 PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cellScanner);
2147 Message result = service.callBlockingMethod(md, controller, param);
2148 long endTime = System.currentTimeMillis();
2149 int processingTime = (int) (endTime - startTime);
2150 int qTime = (int) (startTime - receiveTime);
2151 int totalTime = (int) (endTime - receiveTime);
2152 if (LOG.isTraceEnabled()) {
2153 LOG.trace(CurCall.get().toString() +
2154 ", response " + TextFormat.shortDebugString(result) +
2155 " queueTime: " + qTime +
2156 " processingTime: " + processingTime +
2157 " totalTime: " + totalTime);
2158 }
2159 long requestSize = param.getSerializedSize();
2160 long responseSize = result.getSerializedSize();
2161 metrics.dequeuedCall(qTime);
2162 metrics.processedCall(processingTime);
2163 metrics.totalCall(totalTime);
2164 metrics.receivedRequest(requestSize);
2165 metrics.sentResponse(responseSize);
2166
2167
2168 boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
2169 boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
2170 if (tooSlow || tooLarge) {
2171
2172
2173 StringBuilder buffer = new StringBuilder(256);
2174 buffer.append(md.getName());
2175 buffer.append("(");
2176 buffer.append(param.getClass().getName());
2177 buffer.append(")");
2178 logResponse(new Object[]{param},
2179 md.getName(), buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
2180 status.getClient(), startTime, processingTime, qTime,
2181 responseSize);
2182 }
2183 return new Pair<Message, CellScanner>(result,
2184 controller != null? controller.cellScanner(): null);
2185 } catch (Throwable e) {
2186
2187
2188
2189 if (e instanceof ServiceException) e = e.getCause();
2190
2191
2192 metrics.exception(e);
2193
2194 if (e instanceof LinkageError) throw new DoNotRetryIOException(e);
2195 if (e instanceof IOException) throw (IOException)e;
2196 LOG.error("Unexpected throwable object ", e);
2197 throw new IOException(e.getMessage(), e);
2198 }
2199 }
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215 void logResponse(Object[] params, String methodName, String call, String tag,
2216 String clientAddress, long startTime, int processingTime, int qTime,
2217 long responseSize)
2218 throws IOException {
2219
2220 Map<String, Object> responseInfo = new HashMap<String, Object>();
2221 responseInfo.put("starttimems", startTime);
2222 responseInfo.put("processingtimems", processingTime);
2223 responseInfo.put("queuetimems", qTime);
2224 responseInfo.put("responsesize", responseSize);
2225 responseInfo.put("client", clientAddress);
2226 responseInfo.put("class", serverInstance == null? "": serverInstance.getClass().getSimpleName());
2227 responseInfo.put("method", methodName);
2228 if (params.length == 2 && serverInstance instanceof HRegionServer &&
2229 params[0] instanceof byte[] &&
2230 params[1] instanceof Operation) {
2231
2232
2233 TableName tableName = TableName.valueOf(
2234 HRegionInfo.parseRegionName((byte[]) params[0])[0]);
2235 responseInfo.put("table", tableName.getNameAsString());
2236
2237 responseInfo.putAll(((Operation) params[1]).toMap());
2238
2239 LOG.warn("(operation" + tag + "): " +
2240 MAPPER.writeValueAsString(responseInfo));
2241 } else if (params.length == 1 && serverInstance instanceof HRegionServer &&
2242 params[0] instanceof Operation) {
2243
2244 responseInfo.putAll(((Operation) params[0]).toMap());
2245
2246 LOG.warn("(operation" + tag + "): " +
2247 MAPPER.writeValueAsString(responseInfo));
2248 } else {
2249
2250
2251 responseInfo.put("call", call);
2252 LOG.warn("(response" + tag + "): " + MAPPER.writeValueAsString(responseInfo));
2253 }
2254 }
2255
2256
2257 @Override
2258 public synchronized void stop() {
2259 LOG.info("Stopping server on " + port);
2260 running = false;
2261 if (authTokenSecretMgr != null) {
2262 authTokenSecretMgr.stop();
2263 authTokenSecretMgr = null;
2264 }
2265 listener.interrupt();
2266 listener.doStop();
2267 responder.interrupt();
2268 scheduler.stop();
2269 notifyAll();
2270 }
2271
2272
2273
2274
2275
2276
2277 @Override
2278 public synchronized void join() throws InterruptedException {
2279 while (running) {
2280 wait();
2281 }
2282 }
2283
2284
2285
2286
2287
2288
2289
2290 @Override
2291 public synchronized InetSocketAddress getListenerAddress() {
2292 if (listener == null) {
2293 return null;
2294 }
2295 return listener.getAddress();
2296 }
2297
2298
2299
2300
2301
2302 @Override
2303 public void setErrorHandler(HBaseRPCErrorHandler handler) {
2304 this.errorHandler = handler;
2305 }
2306
2307 @Override
2308 public HBaseRPCErrorHandler getErrorHandler() {
2309 return this.errorHandler;
2310 }
2311
2312
2313
2314
2315 public MetricsHBaseServer getMetrics() {
2316 return metrics;
2317 }
2318
2319 @Override
2320 public void addCallSize(final long diff) {
2321 this.callQueueSize.add(diff);
2322 }
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333 @SuppressWarnings("static-access")
2334 public void authorize(UserGroupInformation user, ConnectionHeader connection, InetAddress addr)
2335 throws AuthorizationException {
2336 if (authorize) {
2337 Class<?> c = getServiceInterface(services, connection.getServiceName());
2338 this.authManager.authorize(user != null ? user : null, c, getConf(), addr);
2339 }
2340 }
2341
2342
2343
2344
2345
2346
2347 private static int NIO_BUFFER_LIMIT = 64 * 1024;
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363 protected long channelWrite(GatheringByteChannel channel, BufferChain bufferChain)
2364 throws IOException {
2365 long count = bufferChain.write(channel, NIO_BUFFER_LIMIT);
2366 if (count > 0) this.metrics.sentBytes(count);
2367 return count;
2368 }
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382 protected int channelRead(ReadableByteChannel channel,
2383 ByteBuffer buffer) throws IOException {
2384
2385 int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
2386 channel.read(buffer) : channelIO(channel, null, buffer);
2387 if (count > 0) {
2388 metrics.receivedBytes(count);
2389 }
2390 return count;
2391 }
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406 private static int channelIO(ReadableByteChannel readCh,
2407 WritableByteChannel writeCh,
2408 ByteBuffer buf) throws IOException {
2409
2410 int originalLimit = buf.limit();
2411 int initialRemaining = buf.remaining();
2412 int ret = 0;
2413
2414 while (buf.remaining() > 0) {
2415 try {
2416 int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
2417 buf.limit(buf.position() + ioSize);
2418
2419 ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);
2420
2421 if (ret < ioSize) {
2422 break;
2423 }
2424
2425 } finally {
2426 buf.limit(originalLimit);
2427 }
2428 }
2429
2430 int nBytes = initialRemaining - buf.remaining();
2431 return (nBytes > 0) ? nBytes : ret;
2432 }
2433
2434
2435
2436
2437
2438
2439
2440 public static RpcCallContext getCurrentCall() {
2441 return CurCall.get();
2442 }
2443
2444 public static boolean isInRpcCallContext() {
2445 return CurCall.get() != null;
2446 }
2447
2448
2449
2450
2451
2452
2453 public static User getRequestUser() {
2454 RpcCallContext ctx = getCurrentCall();
2455 return ctx == null? null: ctx.getRequestUser();
2456 }
2457
2458
2459
2460
2461
2462 public static String getRequestUserName() {
2463 User user = getRequestUser();
2464 return user == null? null: user.getShortName();
2465 }
2466
2467
2468
2469
2470 public static InetAddress getRemoteAddress() {
2471 RpcCallContext ctx = getCurrentCall();
2472 return ctx == null? null: ctx.getRemoteAddress();
2473 }
2474
2475
2476
2477
2478
2479
2480 static BlockingServiceAndInterface getServiceAndInterface(
2481 final List<BlockingServiceAndInterface> services, final String serviceName) {
2482 for (BlockingServiceAndInterface bs : services) {
2483 if (bs.getBlockingService().getDescriptorForType().getName().equals(serviceName)) {
2484 return bs;
2485 }
2486 }
2487 return null;
2488 }
2489
2490
2491
2492
2493
2494
2495 static Class<?> getServiceInterface(
2496 final List<BlockingServiceAndInterface> services,
2497 final String serviceName) {
2498 BlockingServiceAndInterface bsasi =
2499 getServiceAndInterface(services, serviceName);
2500 return bsasi == null? null: bsasi.getServiceInterface();
2501 }
2502
2503
2504
2505
2506
2507
2508 static BlockingService getService(
2509 final List<BlockingServiceAndInterface> services,
2510 final String serviceName) {
2511 BlockingServiceAndInterface bsasi =
2512 getServiceAndInterface(services, serviceName);
2513 return bsasi == null? null: bsasi.getBlockingService();
2514 }
2515
2516 static MonitoredRPCHandler getStatus() {
2517
2518 MonitoredRPCHandler status = RpcServer.MONITORED_RPC.get();
2519 if (status != null) {
2520 return status;
2521 }
2522 status = TaskMonitor.get().createRPCStatus(Thread.currentThread().getName());
2523 status.pause("Waiting for a call");
2524 RpcServer.MONITORED_RPC.set(status);
2525 return status;
2526 }
2527
2528
2529
2530
2531
2532 public static InetAddress getRemoteIp() {
2533 Call call = CurCall.get();
2534 if (call != null && call.connection.socket != null) {
2535 return call.connection.socket.getInetAddress();
2536 }
2537 return null;
2538 }
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550 public static void bind(ServerSocket socket, InetSocketAddress address,
2551 int backlog) throws IOException {
2552 try {
2553 socket.bind(address, backlog);
2554 } catch (BindException e) {
2555 BindException bindException =
2556 new BindException("Problem binding to " + address + " : " +
2557 e.getMessage());
2558 bindException.initCause(e);
2559 throw bindException;
2560 } catch (SocketException e) {
2561
2562
2563 if ("Unresolved address".equals(e.getMessage())) {
2564 throw new UnknownHostException("Invalid hostname for server: " +
2565 address.getHostName());
2566 }
2567 throw e;
2568 }
2569 }
2570
2571 public RpcScheduler getScheduler() {
2572 return scheduler;
2573 }
2574 }