Revert "HBASE-12558 TestHCM.testClusterStatus Unexpected exception, expected<org.apache.hadoop.hbase.regionserver.RegionServerStoppedException> but was<junit.framework.AssertionFailedError> -- ADDED DEBUG"
Accidental commit. Reverting.
This reverts commit 2458be08ed
.
This commit is contained in:
parent
4be2034a26
commit
b8ab1b176c
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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 {
|
|||
* <code>compressor</code>.
|
||||
* @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 <code>codec</code> and/or <code>compressor</code>; 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 <code>cellBlock</code>
|
||||
* @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 <code>cellBlock</code>
|
||||
* @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 <code>m</code> 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;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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.");
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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(
|
||||
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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<Cell> cells = new ArrayList<Cell>();
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<Object>() {
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue