diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
deleted file mode 100644
index 766ad8fa7e8..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.ipc;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.codec.KeyValueCodec;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.util.PoolMap;
-import org.apache.hadoop.io.compress.CompressionCodec;
-
-import java.net.SocketAddress;
-
-/**
- * Provides the basics for a RpcClient implementation like configuration and Logging.
- */
-@InterfaceAudience.Private
-public abstract class AbstractRpcClient implements RpcClient {
- public static final Log LOG = LogFactory.getLog(AbstractRpcClient.class);
-
- protected final Configuration conf;
- protected String clusterId;
- protected final SocketAddress localAddr;
-
- protected UserProvider userProvider;
- protected final IPCUtil ipcUtil;
-
- protected final int minIdleTimeBeforeClose; // if the connection is idle for more than this
- // time (in ms), it will be closed at any moment.
- protected final int maxRetries; //the max. no. of retries for socket connections
- protected final 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 Codec codec;
- protected final CompressionCodec compressor;
- protected final boolean fallbackAllowed;
-
- protected final int connectTO;
- protected final int readTO;
- protected final int writeTO;
-
- /**
- * Construct an IPC client for the cluster clusterId
- *
- * @param conf configuration
- * @param clusterId the cluster id
- * @param localAddr client socket bind address.
- */
- public AbstractRpcClient(Configuration conf, String clusterId, SocketAddress localAddr) {
- this.userProvider = UserProvider.instantiate(conf);
- this.localAddr = localAddr;
- this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
- this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
- this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
- HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
- this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
- this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
- this.ipcUtil = new IPCUtil(conf);
-
- this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes
- this.conf = conf;
- this.codec = getCodec();
- this.compressor = getCompressor(conf);
- this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
- IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
- 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"));
- }
- }
-
- @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(DEFAULT_CODEC_CLASS, KeyValueCodec.class.getCanonicalName());
- }
-
- /**
- * 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);
- }
- }
-
- /**
- * 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 org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or
- * {@link org.apache.hadoop.hbase.util.PoolMap.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 org.apache.hadoop.hbase.ipc.RpcClient}
- * instances should not exceed the operating system's hard limit on the number of
- * connections.
- *
- * @param config configuration
- * @return either a {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#RoundRobin} or
- * {@link org.apache.hadoop.hbase.util.PoolMap.PoolType#ThreadLocal}
- */
- protected static PoolMap.PoolType getPoolType(Configuration config) {
- return PoolMap.PoolType
- .valueOf(config.get(HConstants.HBASE_CLIENT_IPC_POOL_TYPE), PoolMap.PoolType.RoundRobin,
- PoolMap.PoolType.ThreadLocal);
- }
-
- /**
- * Return the pool size specified in the configuration, which is applicable only if
- * the pool type is {@link org.apache.hadoop.hbase.util.PoolMap.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);
- }
-}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
deleted file mode 100644
index df32730fdb3..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import java.io.IOException;
-
-/** A call waiting for a value. */
-@InterfaceAudience.Private
-public 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 Descriptors.MethodDescriptor md;
- final int timeout; // timeout in millisecond for this call; 0 means infinite.
-
- protected Call(int id, final Descriptors.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 = id;
- 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;
- }
-}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
deleted file mode 100644
index a81e5d1ab07..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CallTimeoutException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-import java.io.IOException;
-
-/**
- * Client-side call timeout
- */
-@SuppressWarnings("serial")
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CallTimeoutException extends IOException {
- public CallTimeoutException(final String msg) {
- super(msg);
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
deleted file mode 100644
index a62d4159ef2..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ConnectionId.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.security.User;
-
-import java.net.InetSocketAddress;
-
-/**
- * This class holds the address and the user ticket, etc. The client connections
- * to servers are uniquely identified by
- */
-@InterfaceAudience.Private
-public class ConnectionId {
- final InetSocketAddress address;
- final User ticket;
- private static final int PRIME = 16777619;
- final String serviceName;
-
- public ConnectionId(User ticket, String serviceName, InetSocketAddress address) {
- this.address = address;
- this.ticket = ticket;
- this.serviceName = serviceName;
- }
-
- public String getServiceName() {
- return this.serviceName;
- }
-
- public InetSocketAddress getAddress() {
- return address;
- }
-
- public 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;
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
deleted file mode 100644
index 12f64518336..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServerException.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * 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 class FailedServerException extends HBaseIOException {
- public FailedServerException(String s) {
- super(s);
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServers.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServers.java
deleted file mode 100644
index 16ec16c4378..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/FailedServers.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-import java.net.InetSocketAddress;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-/**
- * A class to manage a list of servers that failed recently.
- */
-@InterfaceAudience.Private
-public class FailedServers {
- private final LinkedList> failedServers = new
- LinkedList>();
- private final int recheckServersTimeout;
-
- public FailedServers(Configuration conf) {
- this.recheckServersTimeout = conf.getInt(
- RpcClient.FAILED_SERVER_EXPIRY_KEY, RpcClient.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;
- }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
deleted file mode 100644
index 2dbb7760501..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-
-import java.net.SocketAddress;
-
-/**
- * Factory to create a {@link org.apache.hadoop.hbase.ipc.RpcClient}
- */
-@InterfaceAudience.Private
-public final class RpcClientFactory {
-
- public static final String CUSTOM_RPC_CLIENT_IMPL_CONF_KEY = "hbase.rpc.client.impl";
-
- /**
- * Private Constructor
- */
- private RpcClientFactory() {
- }
-
- /**
- * Creates a new RpcClient by the class defined in the configuration or falls back to
- * RpcClientImpl
- * @param conf configuration
- * @param clusterId the cluster id
- * @return newly created RpcClient
- */
- public static RpcClient createClient(Configuration conf, String clusterId) {
- return createClient(conf, clusterId, null);
- }
-
- /**
- * Creates a new RpcClient by the class defined in the configuration or falls back to
- * RpcClientImpl
- * @param conf configuration
- * @param clusterId the cluster id
- * @param localAddr client socket bind address.
- * @return newly created RpcClient
- */
- public static RpcClient createClient(Configuration conf, String clusterId,
- SocketAddress localAddr) {
- String rpcClientClass =
- conf.get(CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
- RpcClientImpl.class.getName());
- return ReflectionUtils.instantiateWithCustomCtor(
- rpcClientClass,
- new Class[] { Configuration.class, String.class, SocketAddress.class },
- new Object[] { conf, clusterId, localAddr }
- );
- }
-}
\ No newline at end of file
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
deleted file mode 100644
index c5578a1af8f..00000000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ /dev/null
@@ -1,1383 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.ipc;
-
-import 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;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
-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.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 javax.net.SocketFactory;
-import javax.security.sasl.SaslException;
-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.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;
-
-/**
- * Does RPC against a cluster. Manages connections per regionserver in the cluster.
- *
See HBaseServer
- */
-@InterfaceAudience.Private
-public class RpcClientImpl extends AbstractRpcClient {
- protected final AtomicInteger callIdCnt = new AtomicInteger();
-
- protected final PoolMap connections;
-
- protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
-
- protected final FailedServers failedServers;
-
- protected final SocketFactory socketFactory; // how to create sockets
-
- 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 RpcClientImpl.Connection.CallSender}
- */
- private static class CallFuture {
- final Call call;
- final int priority;
- final Span span;
-
- // We will use this to stop the writer
- final static CallFuture DEATH_PILL = new CallFuture(null, -1, null);
-
- CallFuture(Call call, int priority, Span span) {
- this.call = call;
- this.priority = priority;
- this.span = span;
- }
- }
-
- /** Thread that reads responses and notifies callers. Each connection owns a
- * socket connected to a remote address. Calls are multiplexed through this
- * socket: responses may be delivered out of order. */
- protected class Connection extends Thread {
- private ConnectionHeader header; // connection header
- protected ConnectionId remoteId;
- protected Socket socket = null; // connected socket
- protected DataInputStream in;
- protected DataOutputStream out;
- private Object outLock = new Object();
- private InetSocketAddress server; // server ip:port
- private String serverPrincipal; // server's krb5 principal name
- private AuthMethod authMethod; // authentication method
- private boolean useSasl;
- private Token extends TokenIdentifier> token;
- private HBaseSaslRpcClient saslRpcClient;
- private int reloginMaxBackoff; // max pause before relogin on sasl failure
- private final Codec codec;
- private final CompressionCodec compressor;
-
- // currently active calls
- protected final ConcurrentSkipListMap calls =
- new ConcurrentSkipListMap();
-
- protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();
- protected final CallSender callSender;
-
-
- /**
- * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt),
- * it gets into a java issue: an interruption during a write closes the socket/channel.
- * A way to avoid this is to use a different thread for writing. This way, on interruptions,
- * we either cancel the writes or ignore the answer if the write is already done, but we
- * don't stop the write in the middle.
- * This adds a thread per region server in the client, so it's kept as an option.
- *
- * The implementation is simple: the client threads adds their call to the queue, and then
- * wait for an answer. The CallSender blocks on the queue, and writes the calls one
- * after the other. On interruption, the client cancels its call. The CallSender checks that
- * the call has not been canceled before writing it.
- *
- * When the connection closes, all the calls not yet sent are dismissed. The client thread
- * is notified with an appropriate exception, as if the call was already sent but the answer
- * not yet received.
- *
- */
- private class CallSender extends Thread implements Closeable {
- protected final BlockingQueue callsToWrite;
-
-
- public CallFuture sendCall(Call call, int priority, Span span)
- throws InterruptedException, IOException {
- CallFuture cts = new CallFuture(call, priority, span);
- if (!callsToWrite.offer(cts)) {
- throw new IOException("Can't add the call " + call.id +
- " to the write queue. callsToWrite.size()=" + callsToWrite.size());
- }
- checkIsOpen(); // We check after the put, to be sure that the call we added won't stay
- // in the list while the cleanup was already done.
- return cts;
- }
-
- @Override
- public void close(){
- assert shouldCloseConnection.get();
- callsToWrite.offer(CallFuture.DEATH_PILL);
- // We don't care if we can't add the death pill to the queue: the writer
- // won't be blocked in the 'take', as its queue is full.
- }
-
- CallSender(String name, Configuration conf) {
- int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);
- callsToWrite = new ArrayBlockingQueue(queueSize);
- setDaemon(true);
- setName(name + " - writer");
- }
-
- public void remove(CallFuture cts){
- callsToWrite.remove(cts);
-
- // By removing the call from the expected call list, we make the list smaller, but
- // it means as well that we don't know how many calls we cancelled.
- calls.remove(cts.call.id);
- cts.call.callComplete();
- }
-
- /**
- * Reads the call from the queue, write them on the socket.
- */
- @Override
- public void run() {
- while (!shouldCloseConnection.get()) {
- CallFuture cts = null;
- try {
- cts = callsToWrite.take();
- } catch (InterruptedException e) {
- markClosed(new InterruptedIOException());
- }
-
- if (cts == null || cts == CallFuture.DEATH_PILL) {
- assert shouldCloseConnection.get();
- break;
- }
-
- if (cts.call.done) {
- continue;
- }
-
- if (cts.call.checkAndSetTimeout()) {
- continue;
- }
-
- try {
- Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span);
- } catch (IOException e) {
- LOG.warn("call write error for call #" + cts.call.id + ", message =" + e.getMessage());
- cts.call.setException(e);
- markClosed(e);
- }
- }
-
- cleanup();
- }
-
- /**
- * Cleans the call not yet sent when we finish.
- */
- private void cleanup() {
- assert shouldCloseConnection.get();
-
- IOException ie = new ConnectionClosingException("Connection to " + server + " is closing.");
- while (true) {
- CallFuture cts = callsToWrite.poll();
- if (cts == null) {
- break;
- }
- if (cts.call != null && !cts.call.done) {
- cts.call.setException(ie);
- }
- }
- }
- }
-
- Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
- throws IOException {
- if (remoteId.getAddress().isUnresolved()) {
- throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
- }
- this.server = remoteId.getAddress();
- this.codec = codec;
- this.compressor = compressor;
-
- UserGroupInformation ticket = remoteId.getTicket().getUGI();
- SecurityInfo securityInfo = SecurityInfo.getInfo(remoteId.getServiceName());
- this.useSasl = userProvider.isHBaseSecurityEnabled();
- if (useSasl && securityInfo != null) {
- AuthenticationProtos.TokenIdentifier.Kind tokenKind = securityInfo.getTokenKind();
- if (tokenKind != null) {
- TokenSelector extends TokenIdentifier> tokenSelector =
- tokenHandlers.get(tokenKind);
- if (tokenSelector != null) {
- token = tokenSelector.selectToken(new Text(clusterId),
- ticket.getTokens());
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("No token selector found for type "+tokenKind);
- }
- }
- String serverKey = securityInfo.getServerPrincipal();
- if (serverKey == null) {
- throw new IOException(
- "Can't obtain server Kerberos config key from SecurityInfo");
- }
- serverPrincipal = SecurityUtil.getServerPrincipal(
- conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
- if (LOG.isDebugEnabled()) {
- LOG.debug("RPC Server Kerberos principal name for service="
- + remoteId.getServiceName() + " is " + serverPrincipal);
- }
- }
-
- if (!useSasl) {
- authMethod = AuthMethod.SIMPLE;
- } else if (token != null) {
- authMethod = AuthMethod.DIGEST;
- } else {
- authMethod = AuthMethod.KERBEROS;
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName +
- ", sasl=" + useSasl);
- }
- reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
- this.remoteId = remoteId;
-
- ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
- builder.setServiceName(remoteId.getServiceName());
- UserInformation userInfoPB = getUserInfo(ticket);
- if (userInfoPB != 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