diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index 7f599da693a..f822709c2b1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
-import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -620,7 +619,7 @@ class ConnectionManager {
this.registry = setupRegistry();
retrieveClusterId();
- this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId);
+ this.rpcClient = new RpcClient(this.conf, this.clusterId);
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
// Do we publish the status?
@@ -640,7 +639,7 @@ class ConnectionManager {
@Override
public void newDead(ServerName sn) {
clearCaches(sn);
- rpcClient.cancelConnections(sn);
+ rpcClient.cancelConnections(sn.getHostname(), sn.getPort());
}
}, conf, listenerClass);
}
@@ -784,6 +783,18 @@ class ConnectionManager {
return RegistryFactory.getRegistry(this);
}
+ /**
+ * For tests only.
+ * @param rpcClient Client we should use instead.
+ * @return Previous rpcClient
+ */
+ @VisibleForTesting
+ RpcClient setRpcClient(final RpcClient rpcClient) {
+ RpcClient oldRpcClient = this.rpcClient;
+ this.rpcClient = rpcClient;
+ return oldRpcClient;
+ }
+
/**
* For tests only.
*/
@@ -2325,7 +2336,7 @@ class ConnectionManager {
clusterStatusListener.close();
}
if (rpcClient != null) {
- rpcClient.close();
+ rpcClient.stop();
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
index 6fb2de34c1a..42561206848 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
+import org.apache.hadoop.hbase.ipc.RpcClient.FailedServerException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index b7e7728fece..67e2524d81f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -51,7 +51,7 @@ import com.google.protobuf.Message;
* Utility to help ipc'ing.
*/
@InterfaceAudience.Private
-public class IPCUtil {
+class IPCUtil {
public static final Log LOG = LogFactory.getLog(IPCUtil.class);
/**
* How much we think the decompressor will expand the original compressed content.
@@ -60,7 +60,7 @@ public class IPCUtil {
private final int cellBlockBuildingInitialBufferSize;
private final Configuration conf;
- public IPCUtil(final Configuration conf) {
+ IPCUtil(final Configuration conf) {
super();
this.conf = conf;
this.cellBlockDecompressionMultiplier =
@@ -81,14 +81,14 @@ public class IPCUtil {
* compressor.
* @param codec
* @param compressor
- * @param cellScanner
+ * @Param cellScanner
* @return Null or byte buffer filled with a cellblock filled with passed-in Cells encoded using
* passed in codec and/or compressor; the returned buffer has been
* flipped and is ready for reading. Use limit to find total size.
* @throws IOException
*/
@SuppressWarnings("resource")
- public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
+ ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
final CellScanner cellScanner)
throws IOException {
if (cellScanner == null) return null;
@@ -145,7 +145,7 @@ public class IPCUtil {
* @return CellScanner to work against the content of cellBlock
* @throws IOException
*/
- public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
+ CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
final byte [] cellBlock)
throws IOException {
return createCellScanner(codec, compressor, cellBlock, 0, cellBlock.length);
@@ -159,7 +159,7 @@ public class IPCUtil {
* @return CellScanner to work against the content of cellBlock
* @throws IOException
*/
- public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
+ CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
final byte [] cellBlock, final int offset, final int length)
throws IOException {
// If compressed, decompress it first before passing it on else we will leak compression
@@ -200,7 +200,7 @@ public class IPCUtil {
* @return The passed in Message serialized with delimiter. Return null if m is null
* @throws IOException
*/
- public static ByteBuffer getDelimitedMessageAsByteBuffer(final Message m) throws IOException {
+ static ByteBuffer getDelimitedMessageAsByteBuffer(final Message m) throws IOException {
if (m == null) return null;
int serializedSize = m.getSerializedSize();
int vintSize = CodedOutputStream.computeRawVarint32Size(serializedSize);
@@ -223,7 +223,7 @@ public class IPCUtil {
* @return Total number of bytes written.
* @throws IOException
*/
- public static int write(final OutputStream dos, final Message header, final Message param,
+ static int write(final OutputStream dos, final Message header, final Message param,
final ByteBuffer cellBlock)
throws IOException {
// Must calculate total size and write that first so other side can read it all in in one
@@ -255,7 +255,7 @@ public class IPCUtil {
* @param len
* @throws IOException
*/
- public static void readChunked(final DataInput in, byte[] dest, int offset, int len)
+ static void readChunked(final DataInput in, byte[] dest, int offset, int len)
throws IOException {
int maxRead = 8192;
@@ -265,9 +265,11 @@ public class IPCUtil {
}
/**
+ * @param header
+ * @param body
* @return Size on the wire when the two messages are written with writeDelimitedTo
*/
- public static int getTotalSizeWhenWrittenDelimited(Message ... messages) {
+ static int getTotalSizeWhenWrittenDelimited(Message ... messages) {
int totalSize = 0;
for (Message m: messages) {
if (m == null) continue;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index 4ededd2d4ec..4586e3ebb40 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -1,4 +1,5 @@
/**
+ *
* 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
@@ -15,64 +16,1750 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.hadoop.hbase.ipc;
-import com.google.protobuf.BlockingRpcChannel;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.ConnectException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+import javax.security.sasl.SaslException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.codec.Codec;
+import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+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.protobuf.generated.TracingProtos.RPCTInfo;
+import org.apache.hadoop.hbase.security.AuthMethod;
+import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
+import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
+import org.apache.hadoop.hbase.security.SecurityInfo;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PoolMap;
+import org.apache.hadoop.hbase.util.PoolMap.PoolType;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
+import org.htrace.Span;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
-import java.io.Closeable;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Message.Builder;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
/**
- * Interface for RpcClient implementations so ConnectionManager can handle it.
+ * Does RPC against a cluster. Manages connections per regionserver in the cluster.
+ *
See HBaseServer
*/
-@InterfaceAudience.Private public interface RpcClient extends Closeable {
- public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
- public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
- public final static String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose";
- public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
- "hbase.ipc.client.fallback-to-simple-auth-allowed";
- public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
- public static final String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting";
- public static final String DEFAULT_CODEC_CLASS = "hbase.client.default.rpc.codec";
+@InterfaceAudience.Private
+public class RpcClient {
+ public static final Log LOG = LogFactory.getLog(RpcClient.class);
+ protected final PoolMap connections;
+
+ protected final AtomicInteger callIdCnt = new AtomicInteger();
+ protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
+ final protected Configuration conf;
+ protected final int minIdleTimeBeforeClose; // if the connection is idle for more than this
+ // time (in ms), it will be closed at any moment.
+ final protected int maxRetries; //the max. no. of retries for socket connections
+ final protected long failureSleep; // Time to sleep before retry on failure.
+ protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
+ protected final boolean tcpKeepAlive; // if T then use keepalives
+ protected final FailedServers failedServers;
+ private final Codec codec;
+ private final CompressionCodec compressor;
+ private final IPCUtil ipcUtil;
+
+ protected final SocketFactory socketFactory; // how to create sockets
+ private final int connectTO;
+ private final int readTO;
+ private final int writeTO;
+ protected String clusterId;
+ protected final SocketAddress localAddr;
+
+ private final boolean fallbackAllowed;
+ private UserProvider userProvider;
+
+ final private static String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect";
+ final static int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds
- public final static String SOCKET_TIMEOUT_CONNECT = "hbase.ipc.client.socket.timeout.connect";
/**
* How long we wait when we wait for an answer. It's not the operation time, it's the time
- * we wait when we start to receive an answer, when the remote write starts to send the data.
+ * we wait when we start to receive an answer, when the remote write starts to send the data.
*/
- public final static String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read";
- public final static String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write";
- public final static int DEFAULT_SOCKET_TIMEOUT_CONNECT = 10000; // 10 seconds
- public final static int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds
- public final static int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds
+ final private static String SOCKET_TIMEOUT_READ = "hbase.ipc.client.socket.timeout.read";
+ final static int DEFAULT_SOCKET_TIMEOUT_READ = 20000; // 20 seconds
+
+ final private static String SOCKET_TIMEOUT_WRITE = "hbase.ipc.client.socket.timeout.write";
+ final static int DEFAULT_SOCKET_TIMEOUT_WRITE = 60000; // 60 seconds
// Used by the server, for compatibility with old clients.
// The client in 0.99+ does not ping the server.
final static int PING_CALL_ID = -1;
+ public final static String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
+ public final static int FAILED_SERVER_EXPIRY_DEFAULT = 2000;
+
+ public final static String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose";
+
+ public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
+ "hbase.ipc.client.fallback-to-simple-auth-allowed";
+ public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
+
+ public static final String SPECIFIC_WRITE_THREAD = "hbase.ipc.client.specificThreadForWriting";
+
+ /**
+ * A class to manage a list of servers that failed recently.
+ */
+ static class FailedServers {
+ private final LinkedList> failedServers = new
+ LinkedList>();
+ private final int recheckServersTimeout;
+
+ FailedServers(Configuration conf) {
+ this.recheckServersTimeout = conf.getInt(
+ FAILED_SERVER_EXPIRY_KEY, FAILED_SERVER_EXPIRY_DEFAULT);
+ }
+
+ /**
+ * Add an address to the list of the failed servers list.
+ */
+ public synchronized void addToFailedServers(InetSocketAddress address) {
+ final long expiry = EnvironmentEdgeManager.currentTime() + recheckServersTimeout;
+ failedServers.addFirst(new Pair(expiry, address.toString()));
+ }
+
+ /**
+ * Check if the server should be considered as bad. Clean the old entries of the list.
+ *
+ * @return true if the server is in the failed servers list
+ */
+ public synchronized boolean isFailedServer(final InetSocketAddress address) {
+ if (failedServers.isEmpty()) {
+ return false;
+ }
+
+ final String lookup = address.toString();
+ final long now = EnvironmentEdgeManager.currentTime();
+
+ // iterate, looking for the search entry and cleaning expired entries
+ Iterator> it = failedServers.iterator();
+ while (it.hasNext()) {
+ Pair cur = it.next();
+ if (cur.getFirst() < now) {
+ it.remove();
+ } else {
+ if (lookup.equals(cur.getSecond())) {
+ return true;
+ }
+ }
+ }
+
+ return false;
+ }
+ }
+
+
+ /**
+ * Indicates that we're trying to connect to a already known as dead server. We will want to
+ * retry: we're getting this because the region location was wrong, or because
+ * the server just died, in which case the retry loop will help us to wait for the
+ * regions to recover.
+ */
+ @SuppressWarnings("serial")
+ @InterfaceAudience.Public
+ @InterfaceStability.Evolving
+ public static class FailedServerException extends HBaseIOException {
+ public FailedServerException(String s) {
+ super(s);
+ }
+ }
+
+ /** A call waiting for a value. */
+ protected class Call {
+ final int id; // call id
+ final Message param; // rpc request method param object
+ /**
+ * Optionally has cells when making call. Optionally has cells set on response. Used
+ * passing cells to the rpc and receiving the response.
+ */
+ CellScanner cells;
+ Message response; // value, null if error
+ // The return type. Used to create shell into which we deserialize the response if any.
+ Message responseDefaultType;
+ IOException error; // exception, null if value
+ volatile boolean done; // true when call is done
+ long startTime;
+ final MethodDescriptor md;
+ final int timeout; // timeout in millisecond for this call; 0 means infinite.
+
+ protected Call(final MethodDescriptor md, Message param, final CellScanner cells,
+ final Message responseDefaultType, int timeout) {
+ this.param = param;
+ this.md = md;
+ this.cells = cells;
+ this.startTime = EnvironmentEdgeManager.currentTime();
+ this.responseDefaultType = responseDefaultType;
+ this.id = callIdCnt.getAndIncrement();
+ this.timeout = timeout;
+ }
+
+
+ /**
+ * Check if the call did timeout. Set an exception (includes a notify) if it's the case.
+ * @return true if the call is on timeout, false otherwise.
+ */
+ public boolean checkAndSetTimeout() {
+ if (timeout == 0){
+ return false;
+ }
+
+ long waitTime = EnvironmentEdgeManager.currentTime() - getStartTime();
+ if (waitTime >= timeout) {
+ IOException ie = new CallTimeoutException("Call id=" + id +
+ ", waitTime=" + waitTime + ", operationTimeout=" + timeout + " expired.");
+ setException(ie); // includes a notify
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ public int remainingTime() {
+ if (timeout == 0) {
+ return Integer.MAX_VALUE;
+ }
+
+ int remaining = timeout - (int) (EnvironmentEdgeManager.currentTime() - getStartTime());
+ return remaining > 0 ? remaining : 0;
+ }
+
+ @Override
+ public String toString() {
+ return "callId: " + this.id + " methodName: " + this.md.getName() + " param {" +
+ (this.param != null? ProtobufUtil.getShortTextFormat(this.param): "") + "}";
+ }
+
+ /** Indicate when the call is complete and the
+ * value or error are available. Notifies by default. */
+ protected synchronized void callComplete() {
+ this.done = true;
+ notify(); // notify caller
+ }
+
+ /** Set the exception when there is an error.
+ * Notify the caller the call is done.
+ *
+ * @param error exception thrown by the call; either local or remote
+ */
+ public void setException(IOException error) {
+ this.error = error;
+ callComplete();
+ }
+
+ /**
+ * Set the return value when there is no error.
+ * Notify the caller the call is done.
+ *
+ * @param response return value of the call.
+ * @param cells Can be null
+ */
+ public void setResponse(Message response, final CellScanner cells) {
+ this.response = response;
+ this.cells = cells;
+ callComplete();
+ }
+
+ public long getStartTime() {
+ return this.startTime;
+ }
+ }
+
+ protected final static Map> tokenHandlers =
+ new HashMap>();
+ static {
+ tokenHandlers.put(AuthenticationProtos.TokenIdentifier.Kind.HBASE_AUTH_TOKEN,
+ new AuthenticationTokenSelector());
+ }
+
+ /**
+ * Creates a connection. Can be overridden by a subclass for testing.
+ * @param remoteId - the ConnectionId to use for the connection creation.
+ */
+ protected Connection createConnection(ConnectionId remoteId, final Codec codec,
+ final CompressionCodec compressor)
+ throws IOException {
+ return new Connection(remoteId, codec, compressor);
+ }
+
+ /**
+ * see {@link org.apache.hadoop.hbase.ipc.RpcClient.Connection.CallSender}
+ */
+ private static class CallFuture {
+ final Call call;
+ final int priority;
+ final Span span;
+
+ // We will use this to stop the writer
+ final static CallFuture DEATH_PILL = new CallFuture(null, -1, null);
+
+ CallFuture(Call call, int priority, Span span) {
+ this.call = call;
+ this.priority = priority;
+ this.span = span;
+ }
+ }
+
+ /** Thread that reads responses and notifies callers. Each connection owns a
+ * socket connected to a remote address. Calls are multiplexed through this
+ * socket: responses may be delivered out of order. */
+ protected class Connection extends Thread {
+ private ConnectionHeader header; // connection header
+ protected ConnectionId remoteId;
+ protected Socket socket = null; // connected socket
+ protected DataInputStream in;
+ protected DataOutputStream out;
+ private Object outLock = new Object();
+ private InetSocketAddress server; // server ip:port
+ private String serverPrincipal; // server's krb5 principal name
+ private AuthMethod authMethod; // authentication method
+ private boolean useSasl;
+ private Token extends TokenIdentifier> token;
+ private HBaseSaslRpcClient saslRpcClient;
+ private int reloginMaxBackoff; // max pause before relogin on sasl failure
+ private final Codec codec;
+ private final CompressionCodec compressor;
+
+ // currently active calls
+ protected final ConcurrentSkipListMap calls =
+ new ConcurrentSkipListMap();
+
+ protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();
+ protected final CallSender callSender;
+
+
+ /**
+ * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt),
+ * it gets into a java issue: an interruption during a write closes the socket/channel.
+ * A way to avoid this is to use a different thread for writing. This way, on interruptions,
+ * we either cancel the writes or ignore the answer if the write is already done, but we
+ * don't stop the write in the middle.
+ * This adds a thread per region server in the client, so it's kept as an option.
+ *
+ * The implementation is simple: the client threads adds their call to the queue, and then
+ * wait for an answer. The CallSender blocks on the queue, and writes the calls one
+ * after the other. On interruption, the client cancels its call. The CallSender checks that
+ * the call has not been canceled before writing it.
+ *
+ * When the connection closes, all the calls not yet sent are dismissed. The client thread
+ * is notified with an appropriate exception, as if the call was already sent but the answer
+ * not yet received.
+ *
+ */
+ private class CallSender extends Thread implements Closeable {
+ protected final BlockingQueue callsToWrite;
+
+
+ public CallFuture sendCall(Call call, int priority, Span span)
+ throws InterruptedException, IOException {
+ CallFuture cts = new CallFuture(call, priority, span);
+ if (!callsToWrite.offer(cts)) {
+ throw new IOException("Can't add the call " + call.id +
+ " to the write queue. callsToWrite.size()=" + callsToWrite.size());
+ }
+ checkIsOpen(); // We check after the put, to be sure that the call we added won't stay
+ // in the list while the cleanup was already done.
+ return cts;
+ }
+
+ @Override
+ public void close(){
+ assert shouldCloseConnection.get();
+ callsToWrite.offer(CallFuture.DEATH_PILL);
+ // We don't care if we can't add the death pill to the queue: the writer
+ // won't be blocked in the 'take', as its queue is full.
+ }
+
+ CallSender(String name, Configuration conf) {
+ int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
+ callsToWrite = new ArrayBlockingQueue(queueSize);
+ setDaemon(true);
+ setName(name + " - writer");
+ }
+
+ public void remove(CallFuture cts){
+ callsToWrite.remove(cts);
+
+ // By removing the call from the expected call list, we make the list smaller, but
+ // it means as well that we don't know how many calls we cancelled.
+ calls.remove(cts.call.id);
+ cts.call.callComplete();
+ }
+
+ /**
+ * Reads the call from the queue, write them on the socket.
+ */
+ @Override
+ public void run() {
+ while (!shouldCloseConnection.get()) {
+ CallFuture cts = null;
+ try {
+ cts = callsToWrite.take();
+ } catch (InterruptedException e) {
+ markClosed(new InterruptedIOException());
+ }
+
+ if (cts == null || cts == CallFuture.DEATH_PILL) {
+ assert shouldCloseConnection.get();
+ break;
+ }
+
+ if (cts.call.done) {
+ continue;
+ }
+
+ if (cts.call.checkAndSetTimeout()) {
+ continue;
+ }
+
+ try {
+ Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span);
+ } catch (IOException e) {
+ LOG.warn("call write error for call #" + cts.call.id + ", message =" + e.getMessage());
+ cts.call.setException(e);
+ markClosed(e);
+ }
+ }
+
+ cleanup();
+ }
+
+ /**
+ * Cleans the call not yet sent when we finish.
+ */
+ private void cleanup() {
+ assert shouldCloseConnection.get();
+
+ IOException ie = new ConnectionClosingException("Connection to " + server + " is closing.");
+ while (true) {
+ CallFuture cts = callsToWrite.poll();
+ if (cts == null) {
+ break;
+ }
+ if (cts.call != null && !cts.call.done) {
+ cts.call.setException(ie);
+ }
+ }
+ }
+ }
+
+ Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
+ throws IOException {
+ if (remoteId.getAddress().isUnresolved()) {
+ throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
+ }
+ this.server = remoteId.getAddress();
+ this.codec = codec;
+ this.compressor = compressor;
+
+ UserGroupInformation ticket = remoteId.getTicket().getUGI();
+ SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
+ this.useSasl = userProvider.isHBaseSecurityEnabled();
+ if (useSasl && securityInfo != null) {
+ AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
+ if (tokenKind != null) {
+ TokenSelector extends TokenIdentifier> tokenSelector =
+ tokenHandlers.get(tokenKind);
+ if (tokenSelector != null) {
+ token = tokenSelector.selectToken(new Text(clusterId),
+ ticket.getTokens());
+ } else if (LOG.isDebugEnabled()) {
+ LOG.debug("No token selector found for type "+tokenKind);
+ }
+ }
+ String serverKey = securityInfo.getServerPrincipal();
+ if (serverKey == null) {
+ throw new IOException(
+ "Can't obtain server Kerberos config key from SecurityInfo");
+ }
+ serverPrincipal = SecurityUtil.getServerPrincipal(
+ conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("RPC Server Kerberos principal name for service="
+ + remoteId.getServiceName() + " is " + serverPrincipal);
+ }
+ }
+
+ if (!useSasl) {
+ authMethod = AuthMethod.SIMPLE;
+ } else if (token != null) {
+ authMethod = AuthMethod.DIGEST;
+ } else {
+ authMethod = AuthMethod.KERBEROS;
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
+ ", sasl=" + useSasl);
+ }
+ reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
+ this.remoteId = remoteId;
+
+ ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
+ builder.setServiceName(remoteId.getServiceName());
+ UserInformation userInfoPB;
+ if ((userInfoPB = getUserInfo(ticket)) != null) {
+ builder.setUserInfo(userInfoPB);
+ }
+ if (this.codec != null) {
+ builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
+ }
+ if (this.compressor != null) {
+ builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
+ }
+ this.header = builder.build();
+
+ this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
+ remoteId.getAddress().toString() +
+ ((ticket==null)?" from an unknown user": (" from "
+ + ticket.getUserName())));
+ this.setDaemon(true);
+
+ if (conf.getBoolean(SPECIFIC_WRITE_THREAD, false)) {
+ callSender = new CallSender(getName(), conf);
+ callSender.start();
+ } else {
+ callSender = null;
+ }
+ }
+
+ private UserInformation getUserInfo(UserGroupInformation ugi) {
+ if (ugi == null || authMethod == AuthMethod.DIGEST) {
+ // Don't send user for token auth
+ return null;
+ }
+ UserInformation.Builder userInfoPB = UserInformation.newBuilder();
+ if (authMethod == AuthMethod.KERBEROS) {
+ // Send effective user for Kerberos auth
+ userInfoPB.setEffectiveUser(ugi.getUserName());
+ } else if (authMethod == AuthMethod.SIMPLE) {
+ //Send both effective user and real user for simple auth
+ userInfoPB.setEffectiveUser(ugi.getUserName());
+ if (ugi.getRealUser() != null) {
+ userInfoPB.setRealUser(ugi.getRealUser().getUserName());
+ }
+ }
+ return userInfoPB.build();
+ }
+
+
+
+ protected synchronized void setupConnection() throws IOException {
+ short ioFailures = 0;
+ short timeoutFailures = 0;
+ while (true) {
+ try {
+ this.socket = socketFactory.createSocket();
+ this.socket.setTcpNoDelay(tcpNoDelay);
+ this.socket.setKeepAlive(tcpKeepAlive);
+ if (localAddr != null) {
+ this.socket.bind(localAddr);
+ }
+ NetUtils.connect(this.socket, remoteId.getAddress(), connectTO);
+ this.socket.setSoTimeout(readTO);
+ return;
+ } catch (SocketTimeoutException toe) {
+ /* The max number of retries is 45,
+ * which amounts to 20s*45 = 15 minutes retries.
+ */
+ handleConnectionFailure(timeoutFailures++, maxRetries, toe);
+ } catch (IOException ie) {
+ handleConnectionFailure(ioFailures++, maxRetries, ie);
+ }
+ }
+ }
+
+ protected synchronized void closeConnection() {
+ if (socket == null) {
+ return;
+ }
+
+ // close the current connection
+ try {
+ if (socket.getOutputStream() != null) {
+ socket.getOutputStream().close();
+ }
+ } catch (IOException ignored) { // Can happen if the socket is already closed
+ if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
+ }
+ try {
+ if (socket.getInputStream() != null) {
+ socket.getInputStream().close();
+ }
+ } catch (IOException ignored) { // Can happen if the socket is already closed
+ if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
+ }
+ try {
+ if (socket.getChannel() != null) {
+ socket.getChannel().close();
+ }
+ } catch (IOException ignored) { // Can happen if the socket is already closed
+ if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);
+ }
+ try {
+ socket.close();
+ } catch (IOException e) {
+ LOG.warn("Not able to close a socket", e);
+ }
+
+ // set socket to null so that the next call to setupIOstreams
+ // can start the process of connect all over again.
+ socket = null;
+ }
+
+ /**
+ * Handle connection failures
+ *
+ * If the current number of retries is equal to the max number of retries,
+ * stop retrying and throw the exception; Otherwise backoff N seconds and
+ * try connecting again.
+ *
+ * This Method is only called from inside setupIOstreams(), which is
+ * synchronized. Hence the sleep is synchronized; the locks will be retained.
+ *
+ * @param curRetries current number of retries
+ * @param maxRetries max number of retries allowed
+ * @param ioe failure reason
+ * @throws IOException if max number of retries is reached
+ */
+ private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)
+ throws IOException {
+ closeConnection();
+
+ // throw the exception if the maximum number of retries is reached
+ if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {
+ throw ioe;
+ }
+
+ // otherwise back off and retry
+ try {
+ Thread.sleep(failureSleep);
+ } catch (InterruptedException ie) {
+ ExceptionUtil.rethrowIfInterrupt(ie);
+ }
+
+ LOG.info("Retrying connect to server: " + remoteId.getAddress() +
+ " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
+ " time(s).");
+ }
+
+ /**
+ * @throws IOException if the connection is not open.
+ */
+ private void checkIsOpen() throws IOException {
+ if (shouldCloseConnection.get()) {
+ throw new ConnectionClosingException(getName() + " is closing");
+ }
+ }
+
+ /* wait till someone signals us to start reading RPC response or
+ * it is idle too long, it is marked as to be closed,
+ * or the client is marked as not running.
+ *
+ * @return true if it is time to read a response; false otherwise.
+ */
+ protected synchronized boolean waitForWork() throws InterruptedException {
+ // beware of the concurrent access to the calls list: we can add calls, but as well
+ // remove them.
+ long waitUntil = EnvironmentEdgeManager.currentTime() + minIdleTimeBeforeClose;
+
+ while (true) {
+ if (shouldCloseConnection.get()) {
+ return false;
+ }
+
+ if (!running.get()) {
+ markClosed(new IOException("stopped with " + calls.size() + " pending request(s)"));
+ return false;
+ }
+
+ if (!calls.isEmpty()) {
+ // shouldCloseConnection can be set to true by a parallel thread here. The caller
+ // will need to check anyway.
+ return true;
+ }
+
+ if (EnvironmentEdgeManager.currentTime() >= waitUntil) {
+ // Connection is idle.
+ // We expect the number of calls to be zero here, but actually someone can
+ // adds a call at the any moment, as there is no synchronization between this task
+ // and adding new calls. It's not a big issue, but it will get an exception.
+ markClosed(new IOException(
+ "idle connection closed with " + calls.size() + " pending request(s)"));
+ return false;
+ }
+
+ wait(Math.min(minIdleTimeBeforeClose, 1000));
+ }
+ }
+
+ public InetSocketAddress getRemoteAddress() {
+ return remoteId.getAddress();
+ }
+
+ @Override
+ public void run() {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(getName() + ": starting, connections " + connections.size());
+ }
+
+ try {
+ while (waitForWork()) { // Wait here for work - read or close connection
+ readResponse();
+ }
+ } catch (InterruptedException t) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(getName() + ": interrupted while waiting for call responses");
+ }
+ markClosed(ExceptionUtil.asInterrupt(t));
+ } catch (Throwable t) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(getName() + ": unexpected throwable while waiting for call responses", t);
+ }
+ markClosed(new IOException("Unexpected throwable while waiting call responses", t));
+ }
+
+ close();
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(getName() + ": stopped, connections " + connections.size());
+ }
+ }
+
+ private synchronized void disposeSasl() {
+ if (saslRpcClient != null) {
+ try {
+ saslRpcClient.dispose();
+ saslRpcClient = null;
+ } catch (IOException ioe) {
+ LOG.error("Error disposing of SASL client", ioe);
+ }
+ }
+ }
+
+ private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
+ UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+ UserGroupInformation currentUser =
+ UserGroupInformation.getCurrentUser();
+ UserGroupInformation realUser = currentUser.getRealUser();
+ return authMethod == AuthMethod.KERBEROS &&
+ loginUser != null &&
+ //Make sure user logged in using Kerberos either keytab or TGT
+ loginUser.hasKerberosCredentials() &&
+ // relogin only in case it is the login user (e.g. JT)
+ // or superuser (like oozie).
+ (loginUser.equals(currentUser) || loginUser.equals(realUser));
+ }
+
+ private synchronized boolean setupSaslConnection(final InputStream in2,
+ final OutputStream out2) throws IOException {
+ saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
+ conf.get("hbase.rpc.protection",
+ QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
+ return saslRpcClient.saslConnect(in2, out2);
+ }
+
+ /**
+ * If multiple clients with the same principal try to connect
+ * to the same server at the same time, the server assumes a
+ * replay attack is in progress. This is a feature of kerberos.
+ * In order to work around this, what is done is that the client
+ * backs off randomly and tries to initiate the connection
+ * again.
+ * The other problem is to do with ticket expiry. To handle that,
+ * a relogin is attempted.
+ *
+ * The retry logic is governed by the {@link #shouldAuthenticateOverKrb}
+ * method. In case when the user doesn't have valid credentials, we don't
+ * need to retry (from cache or ticket). In such cases, it is prudent to
+ * throw a runtime exception when we receive a SaslException from the
+ * underlying authentication implementation, so there is no retry from
+ * other high level (for eg, HCM or HBaseAdmin).
+ *
+ */
+ private synchronized void handleSaslConnectionFailure(
+ final int currRetries,
+ final int maxRetries, final Exception ex, final Random rand,
+ final UserGroupInformation user)
+ throws IOException, InterruptedException{
+ user.doAs(new PrivilegedExceptionAction