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