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 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 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() { + @Override + public Object run() throws IOException, InterruptedException { + closeConnection(); + if (shouldAuthenticateOverKrb()) { + if (currRetries < maxRetries) { + if (LOG.isDebugEnabled()) { + LOG.debug("Exception encountered while connecting to " + + "the server : " + ex); + } + //try re-login + if (UserGroupInformation.isLoginKeytabBased()) { + UserGroupInformation.getLoginUser().reloginFromKeytab(); + } else { + UserGroupInformation.getLoginUser().reloginFromTicketCache(); + } + disposeSasl(); + //have granularity of milliseconds + //we are sleeping with the Connection lock held but since this + //connection instance is being used for connecting to the server + //in question, it is okay + Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1)); + return null; + } else { + String msg = "Couldn't setup connection for " + + UserGroupInformation.getLoginUser().getUserName() + + " to " + serverPrincipal; + LOG.warn(msg); + throw (IOException) new IOException(msg).initCause(ex); + } + } else { + LOG.warn("Exception encountered while connecting to " + + "the server : " + ex); + } + if (ex instanceof RemoteException) { + throw (RemoteException)ex; + } + if (ex instanceof SaslException) { + String msg = "SASL authentication failed." + + " The most likely cause is missing or invalid credentials." + + " Consider 'kinit'."; + LOG.fatal(msg, ex); + throw new RuntimeException(msg, ex); + } + throw new IOException(ex); + } + }); + } + + protected synchronized void setupIOstreams() throws IOException { + if (socket != null) { + // The connection is already available. Perfect. + return; + } + + if (shouldCloseConnection.get()){ + throw new ConnectionClosingException("This connection is closing"); + } + + if (failedServers.isFailedServer(remoteId.getAddress())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not trying to connect to " + server + + " this server is in the failed servers list"); + } + IOException e = new FailedServerException( + "This server is in the failed servers list: " + server); + markClosed(e); + close(); + throw e; + } + + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Connecting to " + server); + } + short numRetries = 0; + final short MAX_RETRIES = 5; + Random rand = null; + while (true) { + setupConnection(); + InputStream inStream = NetUtils.getInputStream(socket); + // This creates a socket with a write timeout. This timeout cannot be changed. + OutputStream outStream = NetUtils.getOutputStream(socket, writeTO); + // Write out the preamble -- MAGIC, version, and auth to use. + writeConnectionHeaderPreamble(outStream); + if (useSasl) { + final InputStream in2 = inStream; + final OutputStream out2 = outStream; + UserGroupInformation ticket = remoteId.getTicket().getUGI(); + if (authMethod == AuthMethod.KERBEROS) { + if (ticket != null && ticket.getRealUser() != null) { + ticket = ticket.getRealUser(); + } + } + boolean continueSasl; + if (ticket == null) throw new FatalConnectionException("ticket/user is null"); + try { + continueSasl = ticket.doAs(new PrivilegedExceptionAction() { + @Override + public Boolean run() throws IOException { + return setupSaslConnection(in2, out2); + } + }); + } catch (Exception ex) { + ExceptionUtil.rethrowIfInterrupt(ex); + if (rand == null) { + rand = new Random(); + } + handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket); + continue; + } + if (continueSasl) { + // Sasl connect is successful. Let's set up Sasl i/o streams. + inStream = saslRpcClient.getInputStream(inStream); + outStream = saslRpcClient.getOutputStream(outStream); + } else { + // fall back to simple auth because server told us so. + authMethod = AuthMethod.SIMPLE; + useSasl = false; + } + } + this.in = new DataInputStream(new BufferedInputStream(inStream)); + synchronized (this.outLock) { + this.out = new DataOutputStream(new BufferedOutputStream(outStream)); + } + // Now write out the connection header + writeConnectionHeader(); + + // start the receiver thread after the socket connection has been set up + start(); + return; + } + } catch (Throwable t) { + IOException e = ExceptionUtil.asInterrupt(t); + if (e == null) { + failedServers.addToFailedServers(remoteId.address); + if (t instanceof LinkageError) { + // probably the hbase hadoop version does not match the running hadoop version + e = new DoNotRetryIOException(t); + } else if (t instanceof IOException) { + e = (IOException) t; + } else { + e = new IOException("Could not set up IO Streams to " + server, t); + } + } + markClosed(e); + close(); + throw e; + } + } + + /** + * Write the RPC header: + */ + private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException { + // Assemble the preamble up in a buffer first and then send it. Writing individual elements, + // they are getting sent across piecemeal according to wireshark and then server is messing + // up the reading on occasion (the passed in stream is not buffered yet). + + // Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE + int rpcHeaderLen = HConstants.RPC_HEADER.array().length; + byte [] preamble = new byte [rpcHeaderLen + 2]; + System.arraycopy(HConstants.RPC_HEADER.array(), 0, preamble, 0, rpcHeaderLen); + preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION; + preamble[rpcHeaderLen + 1] = authMethod.code; + outStream.write(preamble); + outStream.flush(); + } + + /** + * Write the connection header. + */ + private synchronized void writeConnectionHeader() throws IOException { + synchronized (this.outLock) { + this.out.writeInt(this.header.getSerializedSize()); + this.header.writeTo(this.out); + this.out.flush(); + } + } + + /** Close the connection. */ + protected synchronized void close() { + if (!shouldCloseConnection.get()) { + LOG.error(getName() + ": the connection is not in the closed state"); + return; + } + + // release the resources + // first thing to do;take the connection out of the connection list + synchronized (connections) { + connections.removeValue(remoteId, this); + } + + // close the streams and therefore the socket + synchronized(this.outLock) { + if (this.out != null) { + IOUtils.closeStream(out); + this.out = null; + } + } + IOUtils.closeStream(in); + this.in = null; + disposeSasl(); + + // log the info + if (LOG.isTraceEnabled()) { + LOG.trace(getName() + ": closing ipc connection to " + server); + } + + cleanupCalls(true); + + if (LOG.isTraceEnabled()) { + LOG.trace(getName() + ": ipc connection to " + server + " closed"); + } + } + + protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException { + TraceScope ts = Trace.continueSpan(span); + try { + writeRequest(call, priority, span); + } finally { + ts.close(); + } + } + + /** + * Initiates a call by sending the parameter to the remote server. + * Note: this is not called from the Connection thread, but by other + * threads. + * @see #readResponse() + */ + private void writeRequest(Call call, final int priority, Span span) throws IOException { + RequestHeader.Builder builder = RequestHeader.newBuilder(); + builder.setCallId(call.id); + if (span != null) { + builder.setTraceInfo( + RPCTInfo.newBuilder().setParentId(span.getSpanId()).setTraceId(span.getTraceId())); + } + builder.setMethodName(call.md.getName()); + builder.setRequestParam(call.param != null); + ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells); + if (cellBlock != null) { + CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); + cellBlockBuilder.setLength(cellBlock.limit()); + builder.setCellBlockMeta(cellBlockBuilder.build()); + } + // Only pass priority if there one. Let zero be same as no priority. + if (priority != 0) builder.setPriority(priority); + RequestHeader header = builder.build(); + + setupIOstreams(); + + // Now we're going to write the call. We take the lock, then check that the connection + // is still valid, and, if so we do the write to the socket. If the write fails, we don't + // know where we stand, we have to close the connection. + checkIsOpen(); + IOException writeException = null; + synchronized (this.outLock) { + if (Thread.interrupted()) throw new InterruptedIOException(); + + calls.put(call.id, call); // We put first as we don't want the connection to become idle. + checkIsOpen(); // Now we're checking that it didn't became idle in between. + + try { + IPCUtil.write(this.out, header, call.param, cellBlock); + } catch (IOException e) { + // We set the value inside the synchronized block, this way the next in line + // won't even try to write + shouldCloseConnection.set(true); + writeException = e; + interrupt(); + } + } + + // We added a call, and may be started the connection close. In both cases, we + // need to notify the reader. + synchronized (this) { + notifyAll(); + } + + // Now that we notified, we can rethrow the exception if any. Otherwise we're good. + if (writeException != null) throw writeException; + } + + /* Receive a response. + * Because only one receiver, so no synchronization on in. + */ + protected void readResponse() { + if (shouldCloseConnection.get()) return; + Call call = null; + boolean expectedCall = false; + try { + // See HBaseServer.Call.setResponse for where we write out the response. + // Total size of the response. Unused. But have to read it in anyways. + int totalSize = in.readInt(); + + // Read the header + ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in); + int id = responseHeader.getCallId(); + call = calls.remove(id); // call.done have to be set before leaving this method + expectedCall = (call != null && !call.done); + if (!expectedCall) { + // So we got a response for which we have no corresponding 'call' here on the client-side. + // We probably timed out waiting, cleaned up all references, and now the server decides + // to return a response. There is nothing we can do w/ the response at this stage. Clean + // out the wire of the response so its out of the way and we can get other responses on + // this connection. + int readSoFar = IPCUtil.getTotalSizeWhenWrittenDelimited(responseHeader); + int whatIsLeftToRead = totalSize - readSoFar; + IOUtils.skipFully(in, whatIsLeftToRead); + return; + } + if (responseHeader.hasException()) { + ExceptionResponse exceptionResponse = responseHeader.getException(); + RemoteException re = createRemoteException(exceptionResponse); + call.setException(re); + if (isFatalConnectionException(exceptionResponse)) { + markClosed(re); + } + } else { + Message value = null; + if (call.responseDefaultType != null) { + Builder builder = call.responseDefaultType.newBuilderForType(); + builder.mergeDelimitedFrom(in); + value = builder.build(); + } + CellScanner cellBlockScanner = null; + if (responseHeader.hasCellBlockMeta()) { + int size = responseHeader.getCellBlockMeta().getLength(); + byte [] cellBlock = new byte[size]; + IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length); + cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock); + } + call.setResponse(value, cellBlockScanner); + } + } catch (IOException e) { + if (expectedCall) call.setException(e); + if (e instanceof SocketTimeoutException) { + // Clean up open calls but don't treat this as a fatal condition, + // since we expect certain responses to not make it by the specified + // {@link ConnectionId#rpcTimeout}. + if (LOG.isTraceEnabled()) LOG.trace("ignored", e); + } else { + // Treat this as a fatal condition and close this connection + markClosed(e); + } + } finally { + cleanupCalls(false); + } + } + + /** + * @return True if the exception is a fatal connection exception. + */ + private boolean isFatalConnectionException(final ExceptionResponse e) { + return e.getExceptionClassName(). + equals(FatalConnectionException.class.getName()); + } + + /** + * @param e exception to be wrapped + * @return RemoteException made from passed e + */ + private RemoteException createRemoteException(final ExceptionResponse e) { + String innerExceptionClassName = e.getExceptionClassName(); + boolean doNotRetry = e.getDoNotRetry(); + return e.hasHostname()? + // If a hostname then add it to the RemoteWithExtrasException + new RemoteWithExtrasException(innerExceptionClassName, + e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry): + new RemoteWithExtrasException(innerExceptionClassName, + e.getStackTrace(), doNotRetry); + } + + protected synchronized void markClosed(IOException e) { + if (e == null) throw new NullPointerException(); + + if (shouldCloseConnection.compareAndSet(false, true)) { + if (LOG.isTraceEnabled()) { + LOG.trace(getName() + ": marking at should close, reason: " + e.getMessage()); + } + if (callSender != null) { + callSender.close(); + } + notifyAll(); + } + } + + + /** + * Cleanup the calls older than a given timeout, in milli seconds. + * @param allCalls true for all calls, false for only the calls in timeout + */ + protected synchronized void cleanupCalls(boolean allCalls) { + Iterator> itor = calls.entrySet().iterator(); + while (itor.hasNext()) { + Call c = itor.next().getValue(); + if (c.done) { + // To catch the calls without timeout that were cancelled. + itor.remove(); + } else if (allCalls) { + long waitTime = EnvironmentEdgeManager.currentTime() - c.getStartTime(); + IOException ie = new ConnectionClosingException("Connection to " + getRemoteAddress() + + " is closing. Call id=" + c.id + ", waitTime=" + waitTime); + c.setException(ie); + itor.remove(); + } else if (c.checkAndSetTimeout()) { + itor.remove(); + } else { + // We expect the call to be ordered by timeout. It may not be the case, but stopping + // at the first valid call allows to be sure that we still have something to do without + // spending too much time by reading the full list. + break; + } + } + } + } + + /** + * Client-side call timeout + */ + @SuppressWarnings("serial") + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static class CallTimeoutException extends IOException { + public CallTimeoutException(final String msg) { + super(msg); + } + } + + /** + * Construct an IPC cluster client whose values are of the {@link Message} class. + * @param conf configuration + * @param clusterId the cluster id + * @param factory socket factory + */ + RpcClient(Configuration conf, String clusterId, SocketFactory factory) { + this(conf, clusterId, factory, null); + } + + /** + * Construct an IPC cluster client whose values are of the {@link Message} class. + * @param conf configuration + * @param clusterId the cluster id + * @param factory socket factory + * @param localAddr client socket bind address + */ + RpcClient(Configuration conf, String clusterId, SocketFactory factory, SocketAddress localAddr) { + this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes + this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0); + this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, + HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true); + this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true); + this.ipcUtil = new IPCUtil(conf); + this.conf = conf; + this.codec = getCodec(); + this.compressor = getCompressor(conf); + this.socketFactory = factory; + this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT; + this.connections = new PoolMap(getPoolType(conf), getPoolSize(conf)); + this.failedServers = new FailedServers(conf); + this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, + IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT); + this.localAddr = localAddr; + this.userProvider = UserProvider.instantiate(conf); + this.connectTO = conf.getInt(SOCKET_TIMEOUT_CONNECT, DEFAULT_SOCKET_TIMEOUT_CONNECT); + this.readTO = conf.getInt(SOCKET_TIMEOUT_READ, DEFAULT_SOCKET_TIMEOUT_READ); + this.writeTO = conf.getInt(SOCKET_TIMEOUT_WRITE, DEFAULT_SOCKET_TIMEOUT_WRITE); + + + // login the server principal (if using secure Hadoop) + if (LOG.isDebugEnabled()) { + LOG.debug("Codec=" + this.codec + ", compressor=" + this.compressor + + ", tcpKeepAlive=" + this.tcpKeepAlive + + ", tcpNoDelay=" + this.tcpNoDelay + + ", connectTO=" + this.connectTO + + ", readTO=" + this.readTO + + ", writeTO=" + this.writeTO + + ", minIdleTimeBeforeClose=" + this.minIdleTimeBeforeClose + + ", maxRetries=" + this.maxRetries + + ", fallbackAllowed=" + this.fallbackAllowed + + ", bind address=" + (this.localAddr != null ? this.localAddr : "null")); + } + } + + /** + * Construct an IPC client for the cluster clusterId with the default SocketFactory + * @param conf configuration + * @param clusterId the cluster id + */ + public RpcClient(Configuration conf, String clusterId) { + this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), null); + } + + /** + * Construct an IPC client for the cluster clusterId with the default SocketFactory + * @param conf configuration + * @param clusterId the cluster id + * @param localAddr client socket bind address. + */ + public RpcClient(Configuration conf, String clusterId, SocketAddress localAddr) { + this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf), localAddr); + } + + /** + * Encapsulate the ugly casting and RuntimeException conversion in private method. + * @return Codec to use on this client. + */ + Codec getCodec() { + // For NO CODEC, "hbase.client.rpc.codec" must be configured with empty string AND + // "hbase.client.default.rpc.codec" also -- because default is to do cell block encoding. + String className = conf.get(HConstants.RPC_CODEC_CONF_KEY, getDefaultCodec(this.conf)); + if (className == null || className.length() == 0) return null; + try { + return (Codec)Class.forName(className).newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed getting codec " + className, e); + } + } + + @VisibleForTesting + public static String getDefaultCodec(final Configuration c) { + // If "hbase.client.default.rpc.codec" is empty string -- you can't set it to null because + // Configuration will complain -- then no default codec (and we'll pb everything). Else + // default is KeyValueCodec + return c.get("hbase.client.default.rpc.codec", KeyValueCodec.class.getCanonicalName()); + } + + /** + * Encapsulate the ugly casting and RuntimeException conversion in private method. + * @param conf configuration + * @return The compressor to use on this client. + */ + private static CompressionCodec getCompressor(final Configuration conf) { + String className = conf.get("hbase.client.rpc.compressor", null); + if (className == null || className.isEmpty()) return null; + try { + return (CompressionCodec)Class.forName(className).newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed getting compressor " + className, e); + } + } + + /** + * Return the pool type specified in the configuration, which must be set to + * either {@link PoolType#RoundRobin} or {@link PoolType#ThreadLocal}, + * otherwise default to the former. + * + * For applications with many user threads, use a small round-robin pool. For + * applications with few user threads, you may want to try using a + * thread-local pool. In any case, the number of {@link RpcClient} instances + * should not exceed the operating system's hard limit on the number of + * connections. + * + * @param config configuration + * @return either a {@link PoolType#RoundRobin} or + * {@link PoolType#ThreadLocal} + */ + protected static PoolType getPoolType(Configuration config) { + return PoolType.valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), + PoolType.RoundRobin, PoolType.ThreadLocal); + } + + /** + * Return the pool size specified in the configuration, which is applicable only if + * the pool type is {@link PoolType#RoundRobin}. + * + * @param config configuration + * @return the maximum pool size + */ + protected static int getPoolSize(Configuration config) { + return config.getInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, 1); + } + + /** Stop all threads related to this client. No further calls may be made + * using this client. */ + public void stop() { + if (LOG.isDebugEnabled()) LOG.debug("Stopping rpc client"); + if (!running.compareAndSet(true, false)) return; + + // wake up all connections + synchronized (connections) { + for (Connection conn : connections.values()) { + conn.interrupt(); + } + } + + // wait until all connections are closed + while (!connections.isEmpty()) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + LOG.info("Interrupted while stopping the client. We still have " + connections.size() + + " connections."); + Thread.currentThread().interrupt(); + return; + } + } + } + + Pair call(PayloadCarryingRpcController pcrc, + MethodDescriptor md, Message param, CellScanner cells, + Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout) + throws InterruptedException, IOException { + return + call(pcrc, md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS); + } + + /** Make a call, passing param, to the IPC server running at + * address which is servicing the protocol protocol, + * with the ticket credentials, returning the value. + * Throws exceptions if there are network problems or if the remote code + * threw an exception. + * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. + * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a + * new Connection each time. + * @return A pair with the Message response and the Cell data (if any). + * @throws InterruptedException + * @throws IOException + */ + Pair call(PayloadCarryingRpcController pcrc, MethodDescriptor md, + Message param, CellScanner cells, + Message returnType, User ticket, InetSocketAddress addr, int callTimeout, int priority) + throws IOException, InterruptedException { + final Call call = new Call(md, param, cells, returnType, callTimeout); + + final Connection connection = getConnection(ticket, call, addr, this.codec, this.compressor); + + final CallFuture cts; + if (connection.callSender != null) { + cts = connection.callSender.sendCall(call, priority, Trace.currentSpan()); + if (pcrc != null) { + pcrc.notifyOnCancel(new RpcCallback() { + @Override + public void run(Object parameter) { + connection.callSender.remove(cts); + } + }); + if (pcrc.isCanceled()) { + // To finish if the call was cancelled before we set the notification (race condition) + call.callComplete(); + return new Pair(call.response, call.cells); + } + } + + } else { + cts = null; + connection.tracedWriteRequest(call, priority, Trace.currentSpan()); + } + + while (!call.done) { + if (call.checkAndSetTimeout()) { + if (cts != null) connection.callSender.remove(cts); + break; + } + if (connection.shouldCloseConnection.get()) { + throw new ConnectionClosingException("Call id=" + call.id + + " on server " + addr + " aborted: connection is closing"); + } + try { + synchronized (call) { + if (call.done) break; + call.wait(Math.min(call.remainingTime(), 1000) + 1); + } + } catch (InterruptedException e) { + call.setException(new InterruptedIOException()); + if (cts != null) connection.callSender.remove(cts); + throw e; + } + } + + if (call.error != null) { + if (call.error instanceof RemoteException) { + call.error.fillInStackTrace(); + throw call.error; + } + // local exception + throw wrapException(addr, call.error); + } + + return new Pair(call.response, call.cells); + } + + + /** + * Take an IOException and the address we were trying to connect to + * and return an IOException with the input exception as the cause. + * The new exception provides the stack trace of the place where + * the exception is thrown and some extra diagnostics information. + * If the exception is ConnectException or SocketTimeoutException, + * return a new one of the same type; Otherwise return an IOException. + * + * @param addr target address + * @param exception the relevant exception + * @return an exception to throw + */ + protected IOException wrapException(InetSocketAddress addr, + IOException exception) { + if (exception instanceof ConnectException) { + //connection refused; include the host:port in the error + return (ConnectException)new ConnectException( + "Call to " + addr + " failed on connection exception: " + exception).initCause(exception); + } else if (exception instanceof SocketTimeoutException) { + return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr + + " failed because " + exception).initCause(exception); + } else if (exception instanceof ConnectionClosingException){ + return (ConnectionClosingException) new ConnectionClosingException( + "Call to " + addr + " failed on local exception: " + exception).initCause(exception); + } else { + return (IOException)new IOException("Call to " + addr + " failed on local exception: " + + exception).initCause(exception); + } + } + + /** + * Interrupt the connections to the given ip:port server. This should be called if the server + * is known as actually dead. This will not prevent current operation to be retried, and, + * depending on their own behavior, they may retry on the same server. This can be a feature, + * for example at startup. In any case, they're likely to get connection refused (if the + * process died) or no route to host: i.e. their next retries should be faster and with a + * safe exception. + */ + public void cancelConnections(String hostname, int port) { + synchronized (connections) { + for (Connection connection : connections.values()) { + if (connection.isAlive() && + connection.getRemoteAddress().getPort() == port && + connection.getRemoteAddress().getHostName().equals(hostname)) { + LOG.info("The server on " + hostname + ":" + port + + " is dead - stopping the connection " + connection.remoteId); + connection.interrupt(); // We're interrupting a Reader. It means we want it to finish. + // This will close the connection as well. + } + } + } + } + + /** + * Get a connection from the pool, or create a new one and add it to the + * pool. Connections to a given host/port are reused. + */ + protected Connection getConnection(User ticket, Call call, InetSocketAddress addr, + final Codec codec, final CompressionCodec compressor) + throws IOException { + if (!running.get()) throw new StoppedRpcClientException(); + Connection connection; + ConnectionId remoteId = + new ConnectionId(ticket, call.md.getService().getName(), addr); + synchronized (connections) { + connection = connections.get(remoteId); + if (connection == null) { + connection = createConnection(remoteId, this.codec, this.compressor); + connections.put(remoteId, connection); + } + } + + return connection; + } + + /** + * This class holds the address and the user ticket, etc. The client connections + * to servers are uniquely identified by + */ + protected static class ConnectionId { + final InetSocketAddress address; + final User ticket; + private static final int PRIME = 16777619; + final String serviceName; + + ConnectionId(User ticket, String serviceName, InetSocketAddress address) { + this.address = address; + this.ticket = ticket; + this.serviceName = serviceName; + } + + String getServiceName() { + return this.serviceName; + } + + InetSocketAddress getAddress() { + return address; + } + + User getTicket() { + return ticket; + } + + @Override + public String toString() { + return this.address.toString() + "/" + this.serviceName + "/" + this.ticket; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ConnectionId) { + ConnectionId id = (ConnectionId) obj; + return address.equals(id.address) && + ((ticket != null && ticket.equals(id.ticket)) || + (ticket == id.ticket)) && + this.serviceName == id.serviceName; + } + return false; + } + + @Override // simply use the default Object#hashcode() ? + public int hashCode() { + int hashcode = (address.hashCode() + + PRIME * (PRIME * this.serviceName.hashCode() ^ + (ticket == null ? 0 : ticket.hashCode()))); + return hashcode; + } + } + + /** + * Make a blocking call. Throws exceptions if there are network problems or if the remote code + * threw an exception. + * @param ticket Be careful which ticket you pass. A new user will mean a new Connection. + * {@link UserProvider#getCurrent()} makes a new instance of User each time so will be a + * new Connection each time. + * @return A pair with the Message response and the Cell data (if any). + */ + Message callBlockingMethod(MethodDescriptor md, PayloadCarryingRpcController pcrc, + Message param, Message returnType, final User ticket, final InetSocketAddress isa) + throws ServiceException { + long startTime = 0; + if (LOG.isTraceEnabled()) { + startTime = EnvironmentEdgeManager.currentTime(); + } + int callTimeout = 0; + CellScanner cells = null; + if (pcrc != null) { + callTimeout = pcrc.getCallTimeout(); + cells = pcrc.cellScanner(); + // Clear it here so we don't by mistake try and these cells processing results. + pcrc.setCellScanner(null); + } + Pair val; + try { + val = call(pcrc, md, param, cells, returnType, ticket, isa, callTimeout, + pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS); + if (pcrc != null) { + // Shove the results into controller so can be carried across the proxy/pb service void. + if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond()); + } else if (val.getSecond() != null) { + throw new ServiceException("Client dropping data on the floor!"); + } + + if (LOG.isTraceEnabled()) { + long callTime = EnvironmentEdgeManager.currentTime() - startTime; + LOG.trace("Call: " + md.getName() + ", callTime: " + callTime + "ms"); + } + return val.getFirst(); + } catch (Throwable e) { + throw new ServiceException(e); + } + } + /** * Creates a "channel" that can be used by a blocking protobuf service. Useful setting up * protobuf blocking stubs. - * * @return A blocking rpc channel that goes via this rpc client instance. */ - public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User user, int rpcTimeout); + public BlockingRpcChannel createBlockingRpcChannel(final ServerName sn, + final User ticket, int defaultOperationTimeout) { + return new BlockingRpcChannelImplementation(this, sn, ticket, defaultOperationTimeout); + } /** - * Interrupt the connections to the given server. This should be called if the server - * is known as actually dead. This will not prevent current operation to be retried, and, - * depending on their own behavior, they may retry on the same server. This can be a feature, - * for example at startup. In any case, they're likely to get connection refused (if the - * process died) or no route to host: i.e. their next retries should be faster and with a - * safe exception. + * Blocking rpc channel that goes via hbase rpc. */ - public void cancelConnections(ServerName sn); + @VisibleForTesting + public static class BlockingRpcChannelImplementation implements BlockingRpcChannel { + private final InetSocketAddress isa; + private final RpcClient rpcClient; + private final User ticket; + private final int defaultOperationTimeout; - /** - * Stop all threads related to this client. No further calls may be made - * using this client. - */ - @Override public void close(); -} \ No newline at end of file + /** + * @param defaultOperationTimeout - the default timeout when no timeout is given + * by the caller. + */ + protected BlockingRpcChannelImplementation(final RpcClient rpcClient, final ServerName sn, + final User ticket, int defaultOperationTimeout) { + this.isa = new InetSocketAddress(sn.getHostname(), sn.getPort()); + this.rpcClient = rpcClient; + this.ticket = ticket; + this.defaultOperationTimeout = defaultOperationTimeout; + } + + @Override + public Message callBlockingMethod(MethodDescriptor md, RpcController controller, + Message param, Message returnType) throws ServiceException { + PayloadCarryingRpcController pcrc; + if (controller != null) { + pcrc = (PayloadCarryingRpcController) controller; + if (!pcrc.hasCallTimeout()){ + pcrc.setCallTimeout(defaultOperationTimeout); + } + } else { + pcrc = new PayloadCarryingRpcController(); + pcrc.setCallTimeout(defaultOperationTimeout); + } + + return this.rpcClient.callBlockingMethod(md, pcrc, param, returnType, this.ticket, this.isa); + } + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index 893b112cf07..4a4dcd892ad 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.ipc.FailedServerException; +import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -316,7 +316,7 @@ public class MetaTableLocator { LOG.debug("Exception connecting to " + sn); } catch (UnknownHostException e) { LOG.debug("Unknown host exception connecting to " + sn); - } catch (FailedServerException e) { + } catch (RpcClient.FailedServerException e) { if (LOG.isDebugEnabled()) { LOG.debug("Server " + sn + " is in failed server list."); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 3936f10f765..1c65a2bce3a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -148,7 +148,7 @@ import com.google.protobuf.TextFormat; * 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. * - * @see RpcClientImpl + * @see RpcClient */ @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX}) @InterfaceStability.Evolving diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 2f6679f71b7..36b322faaba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -64,8 +64,8 @@ import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; -import org.apache.hadoop.hbase.ipc.FailedServerException; import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClient.FailedServerException; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 3c7b769e1b1..41ea09d8c59 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.conf.ConfigurationManager; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager; import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -90,7 +91,6 @@ import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.http.InfoServer; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.ipc.ServerRpcController; @@ -782,11 +782,10 @@ public class HRegionServer extends HasThread implements rsQuotaManager = new RegionServerQuotaManager(this); // Setup RPC client for master communication - rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress( - rpcServices.isa.getAddress(), 0)); + rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress( + rpcServices.isa.getAddress(), 0)); - int storefileRefreshPeriod = conf.getInt( - StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD + int storefileRefreshPeriod = conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD , StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD); if (storefileRefreshPeriod > 0) { this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, this, this); @@ -995,7 +994,7 @@ public class HRegionServer extends HasThread implements this.rssStub = null; } if (this.rpcClient != null) { - this.rpcClient.close(); + this.rpcClient.stop(); } if (this.leases != null) { this.leases.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java index 65483c973ea..f8c093fc980 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerRPCTimeout.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.CoordinatedStateManager; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; @@ -65,7 +65,7 @@ public class TestClientScannerRPCTimeout { @BeforeClass public static void setUpBeforeClass() throws Exception { ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)AbstractRpcClient.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); Configuration conf = TEST_UTIL.getConfiguration(); // Don't report so often so easier to see other rpcs diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index 0fbf21e18a2..4a0d3a3ad65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.net.SocketAddress; import java.net.SocketTimeoutException; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; @@ -36,8 +35,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcClientImpl; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,10 +61,6 @@ public class TestClientTimeouts { @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(SLAVES); - // Set the custom RPC client with random timeouts as the client - TEST_UTIL.getConfiguration().set( - RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, - RandomTimeoutRpcClient.class.getName()); } /** @@ -88,9 +81,7 @@ public class TestClientTimeouts { Connection lastConnection = null; boolean lastFailed = false; int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get(); - RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); - + RpcClient rpcClient = newRandomTimeoutRpcClient(); try { for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) { lastFailed = false; @@ -103,6 +94,13 @@ public class TestClientTimeouts { Connection connection = admin.getConnection(); assertFalse(connection == lastConnection); lastConnection = connection; + // Override the connection's rpc client for timeout testing + RpcClient oldRpcClient = + ((ConnectionManager.HConnectionImplementation)connection).setRpcClient( + rpcClient); + if (oldRpcClient != null) { + oldRpcClient.stop(); + } // run some admin commands HBaseAdmin.checkHBaseAvailable(conf); admin.setBalancerRunning(false, false); @@ -113,8 +111,7 @@ public class TestClientTimeouts { } finally { admin.close(); if (admin.getConnection().isClosed()) { - rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + rpcClient = newRandomTimeoutRpcClient(); } } } @@ -122,36 +119,31 @@ public class TestClientTimeouts { assertFalse(lastFailed); assertTrue(RandomTimeoutBlockingRpcChannel.invokations.get() > initialInvocations); } finally { - rpcClient.close(); + rpcClient.stop(); } } - /** - * Rpc Channel implementation with RandomTimeoutBlockingRpcChannel - */ - public static class RandomTimeoutRpcClient extends RpcClientImpl{ - public RandomTimeoutRpcClient(Configuration conf, String clusterId, SocketAddress localAddr) { - super(conf, clusterId, localAddr); - } - - // Return my own instance, one that does random timeouts - @Override - public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, - User ticket, int rpcTimeout) { - return new RandomTimeoutBlockingRpcChannel(this, sn, ticket, rpcTimeout); - } + private static RpcClient newRandomTimeoutRpcClient() { + return new RpcClient( + TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()) { + // Return my own instance, one that does random timeouts + @Override + public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, + User ticket, int rpcTimeout) { + return new RandomTimeoutBlockingRpcChannel(this, sn, ticket, rpcTimeout); + } + }; } /** * Blocking rpc channel that goes via hbase rpc. */ - static class RandomTimeoutBlockingRpcChannel - extends RpcClientImpl.BlockingRpcChannelImplementation { + static class RandomTimeoutBlockingRpcChannel extends RpcClient.BlockingRpcChannelImplementation { private static final Random RANDOM = new Random(System.currentTimeMillis()); public static final double CHANCE_OF_TIMEOUT = 0.3; private static AtomicInteger invokations = new AtomicInteger(); - RandomTimeoutBlockingRpcChannel(final RpcClientImpl rpcClient, final ServerName sn, + RandomTimeoutBlockingRpcChannel(final RpcClient rpcClient, final ServerName sn, final User ticket, final int rpcTimeout) { super(rpcClient, sn, ticket, rpcTimeout); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java index f5807c202ac..a7e37fd3bbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -97,7 +97,7 @@ public class TestFromClientSideNoCodec { public void testNoCodec() { Configuration c = new Configuration(); c.set("hbase.client.default.rpc.codec", ""); - String codec = AbstractRpcClient.getDefaultCodec(c); + String codec = RpcClient.getDefaultCodec(c); assertTrue(codec == null || codec.length() == 0); } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 82a5c76ddbe..3c992704165 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -394,7 +394,7 @@ public class TestHCM { LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); for (int i = 0; i < 5000; i++) { - rpcClient.cancelConnections(sn); + rpcClient.cancelConnections(sn.getHostname(), sn.getPort()); Thread.sleep(5); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java index 76290fbbf4a..0ea0d46df5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -115,7 +115,7 @@ public class FilterTestingCluster { @BeforeClass public static void setUp() throws Exception { ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); - ((Log4JLogger)AbstractRpcClient.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); TEST_UTIL.startMiniCluster(1); initialize(TEST_UTIL.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java index 961001ff1e5..5fb48ae7aca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java @@ -89,8 +89,7 @@ public class TestDelayedRpc { conf, new FifoRpcScheduler(conf, 1)); rpcServer.start(); - RpcClient rpcClient = RpcClientFactory.createClient( - conf, HConstants.DEFAULT_CLUSTER_ID.toString()); + RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString()); try { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( ServerName.valueOf(rpcServer.getListenerAddress().getHostName(), @@ -119,7 +118,7 @@ public class TestDelayedRpc { assertEquals(UNDELAYED, results.get(1).intValue()); assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED : 0xDEADBEEF); } finally { - rpcClient.close(); + rpcClient.stop(); } } @@ -171,8 +170,7 @@ public class TestDelayedRpc { conf, new FifoRpcScheduler(conf, 1)); rpcServer.start(); - RpcClient rpcClient = RpcClientFactory.createClient( - conf, HConstants.DEFAULT_CLUSTER_ID.toString()); + RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString()); try { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( ServerName.valueOf(rpcServer.getListenerAddress().getHostName(), @@ -202,7 +200,7 @@ public class TestDelayedRpc { log.removeAppender(listAppender); } finally { - rpcClient.close(); + rpcClient.stop(); } } @@ -295,8 +293,7 @@ public class TestDelayedRpc { conf, new FifoRpcScheduler(conf, 1)); rpcServer.start(); - RpcClient rpcClient = RpcClientFactory.createClient( - conf, HConstants.DEFAULT_CLUSTER_ID.toString()); + RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString()); try { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( ServerName.valueOf(rpcServer.getListenerAddress().getHostName(), @@ -326,7 +323,7 @@ public class TestDelayedRpc { } assertTrue(caughtException); } finally { - rpcClient.close(); + rpcClient.stop(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java index 26488cf9477..3d7dddb78d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestHBaseClient.java @@ -36,7 +36,7 @@ public class TestHBaseClient { public void testFailedServer(){ ManualEnvironmentEdge ee = new ManualEnvironmentEdge(); EnvironmentEdgeManager.injectEdge( ee ); - FailedServers fs = new FailedServers(new Configuration()); + RpcClient.FailedServers fs = new RpcClient.FailedServers(new Configuration()); InetSocketAddress ia = InetSocketAddress.createUnresolved("bad", 12); InetSocketAddress ia2 = InetSocketAddress.createUnresolved("bad", 12); // same server as ia diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java index 081b5dd59a7..9ab20e68285 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java @@ -178,7 +178,7 @@ public class TestIPC { @Test public void testNoCodec() throws InterruptedException, IOException { Configuration conf = HBaseConfiguration.create(); - RpcClientImpl client = new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT) { + RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT) { @Override Codec getCodec() { return null; @@ -197,7 +197,7 @@ public class TestIPC { // Silly assertion that the message is in the returned pb. assertTrue(r.getFirst().toString().contains(message)); } finally { - client.close(); + client.stop(); rpcServer.stop(); } } @@ -216,10 +216,10 @@ public class TestIPC { throws IOException, InterruptedException, SecurityException, NoSuchMethodException { Configuration conf = new Configuration(HBaseConfiguration.create()); conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName()); - doSimpleTest(conf, new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT)); + doSimpleTest(conf, new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT)); } - private void doSimpleTest(final Configuration conf, final RpcClientImpl client) + private void doSimpleTest(final Configuration conf, final RpcClient client) throws InterruptedException, IOException { TestRpcServer rpcServer = new TestRpcServer(); List cells = new ArrayList(); @@ -239,7 +239,7 @@ public class TestIPC { } assertEquals(count, index); } finally { - client.close(); + client.stop(); rpcServer.stop(); } } @@ -258,7 +258,7 @@ public class TestIPC { }).when(spyFactory).createSocket(); TestRpcServer rpcServer = new TestRpcServer(); - RpcClientImpl client = new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory); + RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory); try { rpcServer.start(); InetSocketAddress address = rpcServer.getListenerAddress(); @@ -270,7 +270,7 @@ public class TestIPC { LOG.info("Caught expected exception: " + e.toString()); assertTrue(StringUtils.stringifyException(e).contains("Injected fault")); } finally { - client.close(); + client.stop(); rpcServer.stop(); } } @@ -281,7 +281,7 @@ public class TestIPC { RpcScheduler scheduler = spy(new FifoRpcScheduler(CONF, 1)); RpcServer rpcServer = new TestRpcServer(scheduler); verify(scheduler).init((RpcScheduler.Context) anyObject()); - RpcClientImpl client = new RpcClientImpl(CONF, HConstants.CLUSTER_ID_DEFAULT); + RpcClient client = new RpcClient(CONF, HConstants.CLUSTER_ID_DEFAULT); try { rpcServer.start(); verify(scheduler).start(); @@ -312,7 +312,7 @@ public class TestIPC { TestRpcServer rpcServer = new TestRpcServer(); MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo"); EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); - RpcClientImpl client = new RpcClientImpl(conf, HConstants.CLUSTER_ID_DEFAULT); + RpcClient client = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT); KeyValue kv = KeyValueUtil.ensureKeyValue(BIG_CELL); Put p = new Put(kv.getRow()); for (int i = 0; i < cellcount; i++) { @@ -354,7 +354,7 @@ public class TestIPC { LOG.info("Cycled " + cycles + " time(s) with " + cellcount + " cell(s) in " + (System.currentTimeMillis() - startTime) + "ms"); } finally { - client.close(); + client.stop(); rpcServer.stop(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java index cee459f6177..d7b7b47505b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java @@ -112,7 +112,7 @@ public class TestProtoBufRpc { @Test public void testProtoBufRpc() throws Exception { - RpcClient rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT); + RpcClient rpcClient = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT); try { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( ServerName.valueOf(this.isa.getHostName(), this.isa.getPort(), System.currentTimeMillis()), @@ -136,7 +136,7 @@ public class TestProtoBufRpc { } catch (ServiceException e) { } } finally { - rpcClient.close(); + rpcClient.stop(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java index 2419918a34b..ccf98128c5d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; @@ -54,7 +53,7 @@ public class TestHMasterRPCException { CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf); HMaster hm = new HMaster(conf, cp); ServerName sm = hm.getServerName(); - RpcClient rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT); + RpcClient rpcClient = new RpcClient(conf, HConstants.CLUSTER_ID_DEFAULT); try { int i = 0; //retry the RPC a few times; we have seen SocketTimeoutExceptions if we @@ -89,7 +88,7 @@ public class TestHMasterRPCException { } fail(); } finally { - rpcClient.close(); + rpcClient.stop(); } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java index b28a1ef8fce..381079e5b39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/TestSecureRPC.java @@ -35,7 +35,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; @@ -100,8 +99,7 @@ public class TestSecureRPC { Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)), isa, conf, new FifoRpcScheduler(conf, 1)); rpcServer.start(); - RpcClient rpcClient = RpcClientFactory - .createClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString()); + RpcClient rpcClient = new RpcClient(conf, HConstants.DEFAULT_CLUSTER_ID.toString()); try { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel( ServerName.valueOf(rpcServer.getListenerAddress().getHostName(), @@ -117,7 +115,7 @@ public class TestSecureRPC { assertEquals(0xDEADBEEF, results.get(0).intValue()); } finally { - rpcClient.close(); + rpcClient.stop(); } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java index 041e112b51c..abdec5810a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java @@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.RpcServerInterface; @@ -401,7 +400,7 @@ public class TestTokenAuthentication { testuser.doAs(new PrivilegedExceptionAction() { public Object run() throws Exception { Configuration c = server.getConfiguration(); - RpcClient rpcClient = RpcClientFactory.createClient(c, clusterId.toString()); + RpcClient rpcClient = new RpcClient(c, clusterId.toString()); ServerName sn = ServerName.valueOf(server.getAddress().getHostName(), server.getAddress().getPort(), System.currentTimeMillis()); @@ -417,7 +416,7 @@ public class TestTokenAuthentication { String authMethod = response.getAuthMethod(); assertEquals("TOKEN", authMethod); } finally { - rpcClient.close(); + rpcClient.stop(); } return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java index b96fab6e5d8..4bcd1018a3e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.ScannerCallable; -import org.apache.hadoop.hbase.ipc.AbstractRpcClient; +import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; @@ -88,7 +88,7 @@ public class TestFlushSnapshotFromClient { // Uncomment the following lines if more verbosity is needed for // debugging (see HBASE-12285 for details). //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); - //((Log4JLogger)AbstractRpcClient.LOG).getLogger().setLevel(Level.ALL); + //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); setupConf(UTIL.getConfiguration()); UTIL.startMiniCluster(NUM_RS);