org.apache.hadoop.hbase.ipc.RpcServer Maven / Gradle / Ivy
Show all versions of hbase-server Show documentation
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.protobuf.BlockingService;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.CodedOutputStream;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.BindException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.nio.channels.CancelledKeyException;
import java.nio.channels.Channels;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.GatheringByteChannel;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.SelectionKey;
import java.nio.channels.Selector;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
import java.nio.channels.WritableByteChannel;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import javax.security.sasl.Sasl;
import javax.security.sasl.SaslException;
import javax.security.sasl.SaslServer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CallQueueTooBigException;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
import org.apache.hadoop.hbase.client.VersionInfoUtil;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
import org.apache.hadoop.hbase.io.ByteBufferInputStream;
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.AuthMethod;
import org.apache.hadoop.hbase.security.HBasePolicyProvider;
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler;
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
import org.apache.hadoop.hbase.security.SaslStatus;
import org.apache.hadoop.hbase.security.SaslUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.TraceInfo;
import org.codehaus.jackson.map.ObjectMapper;
/**
* An RPC server that hosts protobuf described Services.
*
* An RpcServer instance has a Listener that hosts the socket. Listener has fixed number
* of Readers in an ExecutorPool, 10 by default. The Listener does an accept and then
* round robin a Reader is chosen to do the read. The reader is registered on Selector. Read does
* total read off the channel and the parse from which it makes a Call. The call is wrapped in a
* CallRunner and passed to the scheduler to be run. Reader goes back to see if more to be done
* and loops till done.
*
* Scheduler can be variously implemented but default simple scheduler has handlers to which it
* has given the queues into which calls (i.e. CallRunner instances) are inserted. Handlers run
* taking from the queue. They run the CallRunner#run method on each item gotten from queue
* and keep taking while the server is up.
*
* CallRunner#run executes the call. When done, asks the included Call to put itself on new
* queue for Responder to pull from and return result to client.
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving
public class RpcServer implements RpcServerInterface, ConfigurationObserver {
// LOG is being used in CallRunner and the log level is being changed in tests
public static final Log LOG = LogFactory.getLog(RpcServer.class);
private static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION
= new CallQueueTooBigException();
private final boolean authorize;
private boolean isSecurityEnabled;
public static final byte CURRENT_VERSION = 0;
/**
* Whether we allow a fallback to SIMPLE auth for insecure clients when security is enabled.
*/
public static final String FALLBACK_TO_INSECURE_CLIENT_AUTH =
"hbase.ipc.server.fallback-to-simple-auth-allowed";
/**
* How many calls/handler are allowed in the queue.
*/
static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
/**
* The maximum size that we can hold in the RPC queue
*/
private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
private final CellBlockBuilder cellBlockBuilder;
private static final String AUTH_FAILED_FOR = "Auth failed for ";
private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";
private static final Log AUDITLOG = LogFactory.getLog("SecurityLogger." +
Server.class.getName());
protected SecretManager secretManager;
protected ServiceAuthorizationManager authManager;
/** This is set to Call object before Handler invokes an RPC and ybdie
* after the call returns.
*/
protected static final ThreadLocal CurCall = new ThreadLocal();
/** Keeps MonitoredRPCHandler per handler thread. */
static final ThreadLocal MONITORED_RPC
= new ThreadLocal();
protected final InetSocketAddress bindAddress;
protected int port; // port we listen on
protected InetSocketAddress address; // inet address we listen on
private int readThreads; // number of read threads
protected int maxIdleTime; // the maximum idle time after
// which a client may be
// disconnected
protected int thresholdIdleConnections; // the number of idle
// connections after which we
// will start cleaning up idle
// connections
int maxConnectionsToNuke; // the max number of
// connections to nuke
// during a cleanup
protected MetricsHBaseServer metrics;
protected final Configuration conf;
private int maxQueueSize;
protected int socketSendBufferSize;
protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
protected final boolean tcpKeepAlive; // if T then use keepalives
protected final long purgeTimeout; // in milliseconds
/**
* This flag is used to indicate to sub threads when they should go down. When we call
* {@link #start()}, all threads started will consult this flag on whether they should
* keep going. It is set to false when {@link #stop()} is called.
*/
volatile boolean running = true;
/**
* This flag is set to true after all threads are up and 'running' and the server is then opened
* for business by the call to {@link #start()}.
*/
volatile boolean started = false;
/**
* This is a running count of the size of all outstanding calls by size.
*/
protected final Counter callQueueSize = new Counter();
protected final List connectionList =
Collections.synchronizedList(new LinkedList());
//maintain a list
//of client connections
private Listener listener = null;
protected Responder responder = null;
protected AuthenticationTokenSecretManager authTokenSecretMgr = null;
protected int numConnections = 0;
protected HBaseRPCErrorHandler errorHandler = null;
public static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";
private static final RequestTooBigException REQUEST_TOO_BIG_EXCEPTION =
new RequestTooBigException();
private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
/**
* Minimum allowable timeout (in milliseconds) in rpc request's header. This
* configuration exists to prevent the rpc service regarding this request as timeout immediately.
*/
private static final String MIN_CLIENT_REQUEST_TIMEOUT = "hbase.ipc.min.client.request.timeout";
private static final int DEFAULT_MIN_CLIENT_REQUEST_TIMEOUT = 20;
/** Default value for above params */
public static final int DEFAULT_MAX_REQUEST_SIZE = DEFAULT_MAX_CALLQUEUE_SIZE / 4; // 256M
private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
private static final ObjectMapper MAPPER = new ObjectMapper();
private final int maxRequestSize;
private final int warnResponseTime;
private final int warnResponseSize;
private final int minClientRequestTimeout;
private final Server server;
private final List services;
private final RpcScheduler scheduler;
private UserProvider userProvider;
private final BoundedByteBufferPool reservoir;
private volatile boolean allowFallbackToSimpleAuth;
/**
* Used to get details for scan with a scanner_id
* TODO try to figure out a better way and remove reference from regionserver package later.
*/
private RSRpcServices rsRpcServices;
/**
* Datastructure that holds all necessary to a method invocation and then afterward, carries
* the result.
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
@InterfaceStability.Evolving
public class Call implements RpcCallContext {
protected int id; // the client's call id
protected BlockingService service;
protected MethodDescriptor md;
protected RequestHeader header;
protected Message param; // the parameter passed
// Optional cell data passed outside of protobufs.
protected CellScanner cellScanner;
protected Connection connection; // connection to client
protected long timestamp; // the time received when response is null
// the time served when response is not null
protected int timeout;
protected long startTime;
protected long deadline;// the deadline to handle this call, if exceed we can drop it.
/**
* Chain of buffers to send as response.
*/
protected BufferChain response;
protected Responder responder;
protected long size; // size of current call
protected boolean isError;
protected TraceInfo tinfo;
private ByteBuffer cellBlock = null;
private User user;
private InetAddress remoteAddress;
private long responseCellSize = 0;
private long responseBlockSize = 0;
// cumulative size of serialized exceptions
private long exceptionSize = 0;
private boolean retryImmediatelySupported;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
justification="Can't figure why this complaint is happening... see below")
Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
Message param, CellScanner cellScanner, Connection connection, Responder responder,
long size, TraceInfo tinfo, final InetAddress remoteAddress, int timeout) {
this.id = id;
this.service = service;
this.md = md;
this.header = header;
this.param = param;
this.cellScanner = cellScanner;
this.connection = connection;
this.timestamp = System.currentTimeMillis();
this.response = null;
this.responder = responder;
this.isError = false;
this.size = size;
this.tinfo = tinfo;
this.user = connection == null? null: connection.user; // FindBugs: NP_NULL_ON_SOME_PATH
this.remoteAddress = remoteAddress;
this.retryImmediatelySupported =
connection == null? null: connection.retryImmediatelySupported;
this.timeout = timeout;
this.deadline = this.timeout > 0 ? this.timestamp + this.timeout : Long.MAX_VALUE;
}
/**
* Call is done. Execution happened and we returned results to client. It is now safe to
* cleanup.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="Presume the lock on processing request held by caller is protection enough")
void done() {
if (this.cellBlock != null && reservoir != null) {
// Return buffer to reservoir now we are done with it.
reservoir.putBuffer(this.cellBlock);
this.cellBlock = null;
}
this.connection.decRpcCount(); // Say that we're done with this call.
}
@Override
public String toString() {
return toShortString() + " param: " +
(this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") +
" connection: " + connection.toString();
}
protected RequestHeader getHeader() {
return this.header;
}
public boolean hasPriority() {
return this.header.hasPriority();
}
public int getPriority() {
return this.header.getPriority();
}
/*
* Short string representation without param info because param itself could be huge depends on
* the payload of a command
*/
String toShortString() {
String serviceName = this.connection.service != null ?
this.connection.service.getDescriptorForType().getName() : "null";
return "callId: " + this.id + " service: " + serviceName +
" methodName: " + ((this.md != null) ? this.md.getName() : "n/a") +
" size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) +
" connection: " + connection.toString() +
" deadline: " + deadline;
}
String toTraceString() {
String serviceName = this.connection.service != null ?
this.connection.service.getDescriptorForType().getName() : "";
String methodName = (this.md != null) ? this.md.getName() : "";
return serviceName + "." + methodName;
}
protected synchronized void setSaslTokenResponse(ByteBuffer response) {
this.response = new BufferChain(response);
}
protected synchronized void setResponse(Object m, final CellScanner cells,
Throwable t, String errorMsg) {
if (this.isError) return;
if (t != null) this.isError = true;
BufferChain bc = null;
try {
ResponseHeader.Builder headerBuilder = ResponseHeader.newBuilder();
// Presume it a pb Message. Could be null.
Message result = (Message)m;
// Call id.
headerBuilder.setCallId(this.id);
if (t != null) {
ExceptionResponse.Builder exceptionBuilder = ExceptionResponse.newBuilder();
exceptionBuilder.setExceptionClassName(t.getClass().getName());
exceptionBuilder.setStackTrace(errorMsg);
exceptionBuilder.setDoNotRetry(t instanceof DoNotRetryIOException ||
t instanceof NeedUnmanagedConnectionException);
if (t instanceof RegionMovedException) {
// Special casing for this exception. This is only one carrying a payload.
// Do this instead of build a generic system for allowing exceptions carry
// any kind of payload.
RegionMovedException rme = (RegionMovedException)t;
exceptionBuilder.setHostname(rme.getHostname());
exceptionBuilder.setPort(rme.getPort());
}
// Set the exception as the result of the method invocation.
headerBuilder.setException(exceptionBuilder.build());
}
// Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the
// reservoir when finished. This is hacky and the hack is not contained but benefits are
// high when we can avoid a big buffer allocation on each rpc.
this.cellBlock = cellBlockBuilder.buildCellBlock(this.connection.codec,
this.connection.compressionCodec, cells, reservoir);
if (this.cellBlock != null) {
CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
// Presumes the cellBlock bytebuffer has been flipped so limit has total size in it.
cellBlockBuilder.setLength(this.cellBlock.limit());
headerBuilder.setCellBlockMeta(cellBlockBuilder.build());
}
Message header = headerBuilder.build();
byte[] b = createHeaderAndMessageBytes(result, header);
bc = new BufferChain(ByteBuffer.wrap(b), this.cellBlock);
if (connection.useWrap) {
bc = wrapWithSasl(bc);
}
} catch (IOException e) {
LOG.warn("Exception while creating response " + e);
}
this.response = bc;
}
private byte[] createHeaderAndMessageBytes(Message result, Message header)
throws IOException {
// Organize the response as a set of bytebuffers rather than collect it all together inside
// one big byte array; save on allocations.
int headerSerializedSize = 0, resultSerializedSize = 0, headerVintSize = 0,
resultVintSize = 0;
if (header != null) {
headerSerializedSize = header.getSerializedSize();
headerVintSize = CodedOutputStream.computeRawVarint32Size(headerSerializedSize);
}
if (result != null) {
resultSerializedSize = result.getSerializedSize();
resultVintSize = CodedOutputStream.computeRawVarint32Size(resultSerializedSize);
}
// calculate the total size
int totalSize = headerSerializedSize + headerVintSize
+ (resultSerializedSize + resultVintSize)
+ (this.cellBlock == null ? 0 : this.cellBlock.limit());
// The byte[] should also hold the totalSize of the header, message and the cellblock
byte[] b = new byte[headerSerializedSize + headerVintSize + resultSerializedSize
+ resultVintSize + Bytes.SIZEOF_INT];
// The RpcClient expects the int to be in a format that code be decoded by
// the DataInputStream#readInt(). Hence going with the Bytes.toBytes(int)
// form of writing int.
Bytes.putInt(b, 0, totalSize);
CodedOutputStream cos = CodedOutputStream.newInstance(b, Bytes.SIZEOF_INT,
b.length - Bytes.SIZEOF_INT);
if (header != null) {
cos.writeMessageNoTag(header);
}
if (result != null) {
cos.writeMessageNoTag(result);
}
cos.flush();
cos.checkNoSpaceLeft();
return b;
}
private BufferChain wrapWithSasl(BufferChain bc)
throws IOException {
if (!this.connection.useSasl) return bc;
// Looks like no way around this; saslserver wants a byte array. I have to make it one.
// THIS IS A BIG UGLY COPY.
byte [] responseBytes = bc.getBytes();
byte [] token;
// synchronization may be needed since there can be multiple Handler
// threads using saslServer to wrap responses.
synchronized (connection.saslServer) {
token = connection.saslServer.wrap(responseBytes, 0, responseBytes.length);
}
if (LOG.isTraceEnabled()) {
LOG.trace("Adding saslServer wrapped token of size " + token.length
+ " as call response.");
}
ByteBuffer bbTokenLength = ByteBuffer.wrap(Bytes.toBytes(token.length));
ByteBuffer bbTokenBytes = ByteBuffer.wrap(token);
return new BufferChain(bbTokenLength, bbTokenBytes);
}
@Override
public boolean isClientCellBlockSupported() {
return this.connection != null && this.connection.codec != null;
}
@Override
public long disconnectSince() {
if (!connection.channel.isOpen()) {
return System.currentTimeMillis() - timestamp;
} else {
return -1L;
}
}
@Override
public long getResponseExceptionSize() {
return exceptionSize;
}
@Override
public void incrementResponseExceptionSize(long exSize) {
exceptionSize += exSize;
}
public long getSize() {
return this.size;
}
@Override
public long getResponseCellSize() {
return responseCellSize;
}
@Override
public void incrementResponseCellSize(long cellSize) {
responseCellSize += cellSize;
}
@Override
public long getResponseBlockSize() {
return responseBlockSize;
}
@Override
public void incrementResponseBlockSize(long blockSize) {
responseBlockSize += blockSize;
}
@Override
public long getDeadline() {
return deadline;
}
public synchronized void sendResponseIfReady() throws IOException {
// set param null to reduce memory pressure
this.param = null;
this.responder.doRespond(this);
}
public UserGroupInformation getRemoteUser() {
return connection.ugi;
}
@Override
public User getRequestUser() {
return user;
}
@Override
public String getRequestUserName() {
User user = getRequestUser();
return user == null? null: user.getShortName();
}
@Override
public InetAddress getRemoteAddress() {
return remoteAddress;
}
@Override
public VersionInfo getClientVersionInfo() {
return connection.getVersionInfo();
}
@Override
public boolean isRetryImmediatelySupported() {
return retryImmediatelySupported;
}
}
/** Listens on the socket. Creates jobs for the handler threads*/
private class Listener extends Thread {
private ServerSocketChannel acceptChannel = null; //the accept channel
private Selector selector = null; //the selector that we use for the server
private Reader[] readers = null;
private int currentReader = 0;
private Random rand = new Random();
private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
//-tion (for idle connections) ran
private long cleanupInterval = 10000; //the minimum interval between
//two cleanup runs
private int backlogLength;
private ExecutorService readPool;
public Listener(final String name) throws IOException {
super(name);
backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128);
// Create a new server socket and set to non blocking mode
acceptChannel = ServerSocketChannel.open();
acceptChannel.configureBlocking(false);
// Bind the server socket to the binding addrees (can be different from the default interface)
bind(acceptChannel.socket(), bindAddress, backlogLength);
port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
address = (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
// create a selector;
selector= Selector.open();
readers = new Reader[readThreads];
readPool = Executors.newFixedThreadPool(readThreads,
new ThreadFactoryBuilder().setNameFormat(
"RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
",port=" + port).setDaemon(true)
.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
for (int i = 0; i < readThreads; ++i) {
Reader reader = new Reader();
readers[i] = reader;
readPool.execute(reader);
}
LOG.info(getName() + ": started " + readThreads + " reader(s) listening on port=" + port);
// Register accepts on the server socket with the selector.
acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
this.setName("RpcServer.listener,port=" + port);
this.setDaemon(true);
}
private class Reader implements Runnable {
private volatile boolean adding = false;
private final Selector readSelector;
Reader() throws IOException {
this.readSelector = Selector.open();
}
@Override
public void run() {
try {
doRunLoop();
} finally {
try {
readSelector.close();
} catch (IOException ioe) {
LOG.error(getName() + ": error closing read selector in " + getName(), ioe);
}
}
}
private synchronized void doRunLoop() {
while (running) {
try {
readSelector.select();
while (adding) {
this.wait(1000);
}
Iterator iter = readSelector.selectedKeys().iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
iter.remove();
if (key.isValid()) {
if (key.isReadable()) {
doRead(key);
}
}
}
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping");
} catch (CancelledKeyException e) {
LOG.error(getName() + ": CancelledKeyException in Reader", e);
} catch (IOException ex) {
LOG.info(getName() + ": IOException in Reader", ex);
}
}
}
/**
* This gets reader into the state that waits for the new channel
* to be registered with readSelector. If it was waiting in select()
* the thread will be woken up, otherwise whenever select() is called
* it will return even if there is nothing to read and wait
* in while(adding) for finishAdd call
*/
public void startAdd() {
adding = true;
readSelector.wakeup();
}
public synchronized SelectionKey registerChannel(SocketChannel channel)
throws IOException {
return channel.register(readSelector, SelectionKey.OP_READ);
}
public synchronized void finishAdd() {
adding = false;
this.notify();
}
}
/** cleanup connections from connectionList. Choose a random range
* to scan and also have a limit on the number of the connections
* that will be cleanedup per run. The criteria for cleanup is the time
* for which the connection was idle. If 'force' is true then all
* connections will be looked at for the cleanup.
* @param force all connections will be looked at for cleanup
*/
private void cleanupConnections(boolean force) {
if (force || numConnections > thresholdIdleConnections) {
long currentTime = System.currentTimeMillis();
if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
return;
}
int start = 0;
int end = numConnections - 1;
if (!force) {
start = rand.nextInt(numConnections);
end = rand.nextInt(numConnections);
int temp;
if (end < start) {
temp = start;
start = end;
end = temp;
}
}
int i = start;
int numNuked = 0;
while (i <= end) {
Connection c;
synchronized (connectionList) {
try {
c = connectionList.get(i);
} catch (Exception e) {return;}
}
if (c.timedOut(currentTime)) {
if (LOG.isDebugEnabled())
LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
closeConnection(c);
numNuked++;
end--;
//noinspection UnusedAssignment
c = null;
if (!force && numNuked == maxConnectionsToNuke) break;
}
else i++;
}
lastCleanupRunTime = System.currentTimeMillis();
}
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="selector access is not synchronized; seems fine but concerned changing " +
"it will have per impact")
public void run() {
LOG.info(getName() + ": starting");
while (running) {
SelectionKey key = null;
try {
selector.select(); // FindBugs IS2_INCONSISTENT_SYNC
Iterator iter = selector.selectedKeys().iterator();
while (iter.hasNext()) {
key = iter.next();
iter.remove();
try {
if (key.isValid()) {
if (key.isAcceptable())
doAccept(key);
}
} catch (IOException ignored) {
if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
}
key = null;
}
} catch (OutOfMemoryError e) {
if (errorHandler != null) {
if (errorHandler.checkOOME(e)) {
LOG.info(getName() + ": exiting on OutOfMemoryError");
closeCurrentConnection(key, e);
cleanupConnections(true);
return;
}
} else {
// we can run out of memory if we have too many threads
// log the event and sleep for a minute and give
// some thread(s) a chance to finish
LOG.warn(getName() + ": OutOfMemoryError in server select", e);
closeCurrentConnection(key, e);
cleanupConnections(true);
try {
Thread.sleep(60000);
} catch (InterruptedException ex) {
LOG.debug("Interrupted while sleeping");
return;
}
}
} catch (Exception e) {
closeCurrentConnection(key, e);
}
cleanupConnections(false);
}
LOG.info(getName() + ": stopping");
synchronized (this) {
try {
acceptChannel.close();
selector.close();
} catch (IOException ignored) {
if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
}
selector= null;
acceptChannel= null;
// clean up all connections
while (!connectionList.isEmpty()) {
closeConnection(connectionList.remove(0));
}
}
}
private void closeCurrentConnection(SelectionKey key, Throwable e) {
if (key != null) {
Connection c = (Connection)key.attachment();
if (c != null) {
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": disconnecting client " + c.getHostAddress() +
(e != null ? " on error " + e.getMessage() : ""));
}
closeConnection(c);
key.attach(null);
}
}
}
InetSocketAddress getAddress() {
return address;
}
void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
Connection c;
ServerSocketChannel server = (ServerSocketChannel) key.channel();
SocketChannel channel;
while ((channel = server.accept()) != null) {
try {
channel.configureBlocking(false);
channel.socket().setTcpNoDelay(tcpNoDelay);
channel.socket().setKeepAlive(tcpKeepAlive);
} catch (IOException ioe) {
channel.close();
throw ioe;
}
Reader reader = getReader();
try {
reader.startAdd();
SelectionKey readKey = reader.registerChannel(channel);
c = getConnection(channel, System.currentTimeMillis());
readKey.attach(c);
synchronized (connectionList) {
connectionList.add(numConnections, c);
numConnections++;
}
if (LOG.isDebugEnabled())
LOG.debug(getName() + ": connection from " + c.toString() +
"; # active connections: " + numConnections);
} finally {
reader.finishAdd();
}
}
}
void doRead(SelectionKey key) throws InterruptedException {
int count;
Connection c = (Connection) key.attachment();
if (c == null) {
return;
}
c.setLastContact(System.currentTimeMillis());
try {
count = c.readAndProcess();
if (count > 0) {
c.setLastContact(System.currentTimeMillis());
}
} catch (InterruptedException ieo) {
throw ieo;
} catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": Caught exception while reading:", e);
}
count = -1; //so that the (count < 0) block is executed
}
if (count < 0) {
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": DISCONNECTING client " + c.toString() +
" because read count=" + count +
". Number of active connections: " + numConnections);
}
closeConnection(c);
}
}
synchronized void doStop() {
if (selector != null) {
selector.wakeup();
Thread.yield();
}
if (acceptChannel != null) {
try {
acceptChannel.socket().close();
} catch (IOException e) {
LOG.info(getName() + ": exception in closing listener socket. " + e);
}
}
readPool.shutdownNow();
}
// The method that will return the next reader to work with
// Simplistic implementation of round robin for now
Reader getReader() {
currentReader = (currentReader + 1) % readers.length;
return readers[currentReader];
}
}
// Sends responses of RPC back to clients.
protected class Responder extends Thread {
private final Selector writeSelector;
private final Set writingCons =
Collections.newSetFromMap(new ConcurrentHashMap());
Responder() throws IOException {
this.setName("RpcServer.responder");
this.setDaemon(true);
this.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER);
writeSelector = Selector.open(); // create a selector
}
@Override
public void run() {
LOG.info(getName() + ": starting");
try {
doRunLoop();
} finally {
LOG.info(getName() + ": stopping");
try {
writeSelector.close();
} catch (IOException ioe) {
LOG.error(getName() + ": couldn't close write selector", ioe);
}
}
}
/**
* Take the list of the connections that want to write, and register them
* in the selector.
*/
private void registerWrites() {
Iterator it = writingCons.iterator();
while (it.hasNext()) {
Connection c = it.next();
it.remove();
SelectionKey sk = c.channel.keyFor(writeSelector);
try {
if (sk == null) {
try {
c.channel.register(writeSelector, SelectionKey.OP_WRITE, c);
} catch (ClosedChannelException e) {
// ignore: the client went away.
if (LOG.isTraceEnabled()) LOG.trace("ignored", e);
}
} else {
sk.interestOps(SelectionKey.OP_WRITE);
}
} catch (CancelledKeyException e) {
// ignore: the client went away.
if (LOG.isTraceEnabled()) LOG.trace("ignored", e);
}
}
}
/**
* Add a connection to the list that want to write,
*/
public void registerForWrite(Connection c) {
if (writingCons.add(c)) {
writeSelector.wakeup();
}
}
private void doRunLoop() {
long lastPurgeTime = 0; // last check for old calls.
while (running) {
try {
registerWrites();
int keyCt = writeSelector.select(purgeTimeout);
if (keyCt == 0) {
continue;
}
Set keys = writeSelector.selectedKeys();
Iterator iter = keys.iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
iter.remove();
try {
if (key.isValid() && key.isWritable()) {
doAsyncWrite(key);
}
} catch (IOException e) {
LOG.debug(getName() + ": asyncWrite", e);
}
}
lastPurgeTime = purge(lastPurgeTime);
} catch (OutOfMemoryError e) {
if (errorHandler != null) {
if (errorHandler.checkOOME(e)) {
LOG.info(getName() + ": exiting on OutOfMemoryError");
return;
}
} else {
//
// we can run out of memory if we have too many threads
// log the event and sleep for a minute and give
// some thread(s) a chance to finish
//
LOG.warn(getName() + ": OutOfMemoryError in server select", e);
try {
Thread.sleep(60000);
} catch (InterruptedException ex) {
LOG.debug("Interrupted while sleeping");
return;
}
}
} catch (Exception e) {
LOG.warn(getName() + ": exception in Responder " +
StringUtils.stringifyException(e), e);
}
}
LOG.info(getName() + ": stopped");
}
/**
* If there were some calls that have not been sent out for a
* long time, we close the connection.
* @return the time of the purge.
*/
private long purge(long lastPurgeTime) {
long now = System.currentTimeMillis();
if (now < lastPurgeTime + purgeTimeout) {
return lastPurgeTime;
}
ArrayList conWithOldCalls = new ArrayList();
// get the list of channels from list of keys.
synchronized (writeSelector.keys()) {
for (SelectionKey key : writeSelector.keys()) {
Connection connection = (Connection) key.attachment();
if (connection == null) {
throw new IllegalStateException("Coding error: SelectionKey key without attachment.");
}
Call call = connection.responseQueue.peekFirst();
if (call != null && now > call.timestamp + purgeTimeout) {
conWithOldCalls.add(call.connection);
}
}
}
// Seems safer to close the connection outside of the synchronized loop...
for (Connection connection : conWithOldCalls) {
closeConnection(connection);
}
return now;
}
private void doAsyncWrite(SelectionKey key) throws IOException {
Connection connection = (Connection) key.attachment();
if (connection == null) {
throw new IOException("doAsyncWrite: no connection");
}
if (key.channel() != connection.channel) {
throw new IOException("doAsyncWrite: bad channel");
}
if (processAllResponses(connection)) {
try {
// We wrote everything, so we don't need to be told when the socket is ready for
// write anymore.
key.interestOps(0);
} catch (CancelledKeyException e) {
/* The Listener/reader might have closed the socket.
* We don't explicitly cancel the key, so not sure if this will
* ever fire.
* This warning could be removed.
*/
LOG.warn("Exception while changing ops : " + e);
}
}
}
/**
* Process the response for this call. You need to have the lock on
* {@link org.apache.hadoop.hbase.ipc.RpcServer.Connection#responseWriteLock}
*
* @param call the call
* @return true if we proceed the call fully, false otherwise.
* @throws IOException
*/
private boolean processResponse(final Call call) throws IOException {
boolean error = true;
try {
// Send as much data as we can in the non-blocking fashion
long numBytes = channelWrite(call.connection.channel, call.response);
if (numBytes < 0) {
throw new HBaseIOException("Error writing on the socket " +
"for the call:" + call.toShortString());
}
error = false;
} finally {
if (error) {
LOG.debug(getName() + call.toShortString() + ": output error -- closing");
closeConnection(call.connection);
}
}
if (!call.response.hasRemaining()) {
call.done();
return true;
} else {
return false; // Socket can't take more, we will have to come back.
}
}
/**
* Process all the responses for this connection
*
* @return true if all the calls were processed or that someone else is doing it.
* false if there * is still some work to do. In this case, we expect the caller to
* delay us.
* @throws IOException
*/
private boolean processAllResponses(final Connection connection) throws IOException {
// We want only one writer on the channel for a connection at a time.
connection.responseWriteLock.lock();
try {
for (int i = 0; i < 20; i++) {
// protection if some handlers manage to need all the responder
Call call = connection.responseQueue.pollFirst();
if (call == null) {
return true;
}
if (!processResponse(call)) {
connection.responseQueue.addFirst(call);
return false;
}
}
} finally {
connection.responseWriteLock.unlock();
}
return connection.responseQueue.isEmpty();
}
//
// Enqueue a response from the application.
//
void doRespond(Call call) throws IOException {
boolean added = false;
// If there is already a write in progress, we don't wait. This allows to free the handlers
// immediately for other tasks.
if (call.connection.responseQueue.isEmpty() && call.connection.responseWriteLock.tryLock()) {
try {
if (call.connection.responseQueue.isEmpty()) {
// If we're alone, we can try to do a direct call to the socket. It's
// an optimisation to save on context switches and data transfer between cores..
if (processResponse(call)) {
return; // we're done.
}
// Too big to fit, putting ahead.
call.connection.responseQueue.addFirst(call);
added = true; // We will register to the selector later, outside of the lock.
}
} finally {
call.connection.responseWriteLock.unlock();
}
}
if (!added) {
call.connection.responseQueue.addLast(call);
}
call.responder.registerForWrite(call.connection);
// set the serve time when the response has to be sent later
call.timestamp = System.currentTimeMillis();
}
}
/** Reads calls from a connection and queues them for handling. */
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="VO_VOLATILE_INCREMENT",
justification="False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")
public class Connection {
// If initial preamble with version and magic has been read or not.
private boolean connectionPreambleRead = false;
// If the connection header has been read or not.
private boolean connectionHeaderRead = false;
protected SocketChannel channel;
private ByteBuffer data;
private ByteBuffer dataLengthBuffer;
private ByteBuffer preambleBuffer;
protected final ConcurrentLinkedDeque responseQueue = new ConcurrentLinkedDeque();
private final Lock responseWriteLock = new ReentrantLock();
private Counter rpcCount = new Counter(); // number of outstanding rpcs
private long lastContact;
private InetAddress addr;
protected Socket socket;
// Cache the remote host & port info so that even if the socket is
// disconnected, we can say where it used to connect to.
protected String hostAddress;
protected int remotePort;
ConnectionHeader connectionHeader;
/**
* Codec the client asked use.
*/
private Codec codec;
/**
* Compression codec the client asked us use.
*/
private CompressionCodec compressionCodec;
BlockingService service;
private AuthMethod authMethod;
private boolean saslContextEstablished;
private boolean skipInitialSaslHandshake;
private ByteBuffer unwrappedData;
// When is this set? FindBugs wants to know! Says NP
private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
boolean useSasl;
SaslServer saslServer;
private boolean useWrap = false;
// Fake 'call' for failed authorization response
private static final int AUTHORIZATION_FAILED_CALLID = -1;
private final Call authFailedCall = new Call(AUTHORIZATION_FAILED_CALLID, null, null, null,
null, null, this, null, 0, null, null, 0);
private ByteArrayOutputStream authFailedResponse =
new ByteArrayOutputStream();
// Fake 'call' for SASL context setup
private static final int SASL_CALLID = -33;
private final Call saslCall = new Call(SASL_CALLID, null, null, null, null, null, this, null,
0, null, null, 0);
// was authentication allowed with a fallback to simple auth
private boolean authenticatedWithFallback;
private boolean retryImmediatelySupported = false;
public UserGroupInformation attemptingUser = null; // user name before auth
protected User user = null;
protected UserGroupInformation ugi = null;
public Connection(SocketChannel channel, long lastContact) {
this.channel = channel;
this.lastContact = lastContact;
this.data = null;
this.dataLengthBuffer = ByteBuffer.allocate(4);
this.socket = channel.socket();
this.addr = socket.getInetAddress();
if (addr == null) {
this.hostAddress = "*Unknown*";
} else {
this.hostAddress = addr.getHostAddress();
}
this.remotePort = socket.getPort();
if (socketSendBufferSize != 0) {
try {
socket.setSendBufferSize(socketSendBufferSize);
} catch (IOException e) {
LOG.warn("Connection: unable to set socket send buffer size to " +
socketSendBufferSize);
}
}
}
@Override
public String toString() {
return getHostAddress() + ":" + remotePort;
}
public String getHostAddress() {
return hostAddress;
}
public InetAddress getHostInetAddress() {
return addr;
}
public int getRemotePort() {
return remotePort;
}
public void setLastContact(long lastContact) {
this.lastContact = lastContact;
}
public VersionInfo getVersionInfo() {
if (connectionHeader.hasVersionInfo()) {
return connectionHeader.getVersionInfo();
}
return null;
}
/* Return true if the connection has no outstanding rpc */
private boolean isIdle() {
return rpcCount.get() == 0;
}
/* Decrement the outstanding RPC count */
protected void decRpcCount() {
rpcCount.decrement();
}
/* Increment the outstanding RPC count */
protected void incRpcCount() {
rpcCount.increment();
}
protected boolean timedOut(long currentTime) {
return isIdle() && currentTime - lastContact > maxIdleTime;
}
private UserGroupInformation getAuthorizedUgi(String authorizedId)
throws IOException {
UserGroupInformation authorizedUgi;
if (authMethod == AuthMethod.DIGEST) {
TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
secretManager);
authorizedUgi = tokenId.getUser();
if (authorizedUgi == null) {
throw new AccessDeniedException(
"Can't retrieve username from tokenIdentifier.");
}
authorizedUgi.addTokenIdentifier(tokenId);
} else {
authorizedUgi = UserGroupInformation.createRemoteUser(authorizedId);
}
authorizedUgi.setAuthenticationMethod(authMethod.authenticationMethod.getAuthMethod());
return authorizedUgi;
}
private void saslReadAndProcess(ByteBuffer saslToken) throws IOException,
InterruptedException {
if (saslContextEstablished) {
if (LOG.isTraceEnabled())
LOG.trace("Have read input token of size " + saslToken.limit()
+ " for processing by saslServer.unwrap()");
if (!useWrap) {
processOneRpc(saslToken);
} else {
byte[] b = saslToken.array();
byte [] plaintextData = saslServer.unwrap(b, saslToken.position(), saslToken.limit());
processUnwrappedData(plaintextData);
}
} else {
byte[] replyToken;
try {
if (saslServer == null) {
switch (authMethod) {
case DIGEST:
if (secretManager == null) {
throw new AccessDeniedException(
"Server is not configured to do DIGEST authentication.");
}
saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
.getMechanismName(), null, SaslUtil.SASL_DEFAULT_REALM,
HBaseSaslRpcServer.getSaslProps(), new SaslDigestCallbackHandler(
secretManager, this));
break;
default:
UserGroupInformation current = UserGroupInformation.getCurrentUser();
String fullName = current.getUserName();
if (LOG.isDebugEnabled()) {
LOG.debug("Kerberos principal name is " + fullName);
}
final String names[] = SaslUtil.splitKerberosName(fullName);
if (names.length != 3) {
throw new AccessDeniedException(
"Kerberos principal name does NOT have the expected "
+ "hostname part: " + fullName);
}
current.doAs(new PrivilegedExceptionAction