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