HBASE-25561 Added ignored test for async connection that runs retries just so can check how long it takes and that retrying is happening (#2942)
This commit is contained in:
parent
3bf2ef4259
commit
763de9a085
|
@ -30,6 +30,7 @@ import java.util.Random;
|
||||||
import java.util.SortedMap;
|
import java.util.SortedMap;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
@ -104,6 +105,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpeci
|
||||||
/**
|
/**
|
||||||
* Test client behavior w/o setting up a cluster.
|
* Test client behavior w/o setting up a cluster.
|
||||||
* Mock up cluster emissions.
|
* Mock up cluster emissions.
|
||||||
|
* See below for a method that tests retries/timeouts currently commented out.
|
||||||
*/
|
*/
|
||||||
@Category({ClientTests.class, SmallTests.class})
|
@Category({ClientTests.class, SmallTests.class})
|
||||||
public class TestClientNoCluster extends Configured implements Tool {
|
public class TestClientNoCluster extends Configured implements Tool {
|
||||||
|
@ -114,6 +116,12 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TestClientNoCluster.class);
|
private static final Logger LOG = LoggerFactory.getLogger(TestClientNoCluster.class);
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
|
/**
|
||||||
|
* A server that does not exist. I've changed the server in the below to 'localhost' so we
|
||||||
|
* have a servername that resolves -- otherwise, we just fail on server name lookup with
|
||||||
|
* UnknownHost... With localhost, was able to reproduce stack traces that looked like production
|
||||||
|
* stack traces. Was useful figuring out how retry/timeouts are functioning.
|
||||||
|
*/
|
||||||
public static final ServerName META_SERVERNAME =
|
public static final ServerName META_SERVERNAME =
|
||||||
ServerName.valueOf("meta.example.org", 16010, 12345);
|
ServerName.valueOf("meta.example.org", 16010, 12345);
|
||||||
|
|
||||||
|
@ -149,8 +157,7 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove the @Ignore to try out timeout and retry asettings
|
* Remove the @Ignore to try out timeout and retry settings
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
@Ignore
|
@Ignore
|
||||||
@Test
|
@Test
|
||||||
|
@ -158,6 +165,7 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
Configuration localConfig = HBaseConfiguration.create(this.conf);
|
Configuration localConfig = HBaseConfiguration.create(this.conf);
|
||||||
// This override mocks up our exists/get call to throw a RegionServerStoppedException.
|
// This override mocks up our exists/get call to throw a RegionServerStoppedException.
|
||||||
localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName());
|
localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName());
|
||||||
|
// localConfig.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 7);
|
||||||
Connection connection = ConnectionFactory.createConnection(localConfig);
|
Connection connection = ConnectionFactory.createConnection(localConfig);
|
||||||
Table table = connection.getTable(TableName.META_TABLE_NAME);
|
Table table = connection.getTable(TableName.META_TABLE_NAME);
|
||||||
Throwable t = null;
|
Throwable t = null;
|
||||||
|
@ -180,6 +188,34 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
assertTrue(t != null);
|
assertTrue(t != null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove the @Ignore to try out timeout and retry settings
|
||||||
|
*/
|
||||||
|
// @Ignore
|
||||||
|
@Test
|
||||||
|
public void testAsyncTimeoutAndRetries()
|
||||||
|
throws IOException, ExecutionException, InterruptedException {
|
||||||
|
Configuration localConfig = HBaseConfiguration.create(this.conf);
|
||||||
|
localConfig.set(ConnectionFactory.HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
|
||||||
|
RpcTimeoutAsyncConnection.class.getName());
|
||||||
|
localConfig.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 9);
|
||||||
|
AsyncConnection connection = ConnectionFactory.createAsyncConnection(localConfig).get();
|
||||||
|
AsyncTable table = connection.getTable(TableName.META_TABLE_NAME);
|
||||||
|
Throwable t = null;
|
||||||
|
LOG.info("Start");
|
||||||
|
try {
|
||||||
|
// An exists call turns into a get w/ a flag.
|
||||||
|
table.exists(new Get(Bytes.toBytes("abc"))).get();
|
||||||
|
} catch (Throwable throwable) {
|
||||||
|
// What to catch?
|
||||||
|
t = throwable;
|
||||||
|
} finally {
|
||||||
|
connection.close();
|
||||||
|
}
|
||||||
|
LOG.info("Stop");
|
||||||
|
assertTrue(t != null);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that operation timeout prevails over rpc default timeout and retries, etc.
|
* Test that operation timeout prevails over rpc default timeout and retries, etc.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
@ -331,8 +367,7 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
/**
|
/**
|
||||||
* Override to check we are setting rpc timeout right.
|
* Override to check we are setting rpc timeout right.
|
||||||
*/
|
*/
|
||||||
static class RpcTimeoutConnection
|
static class RpcTimeoutConnection extends ConnectionImplementation {
|
||||||
extends ConnectionImplementation {
|
|
||||||
final ClientService.BlockingInterface stub;
|
final ClientService.BlockingInterface stub;
|
||||||
|
|
||||||
RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user)
|
RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user)
|
||||||
|
@ -343,7 +378,7 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
try {
|
try {
|
||||||
Mockito.when(stub.get((RpcController)Mockito.any(),
|
Mockito.when(stub.get((RpcController)Mockito.any(),
|
||||||
(ClientProtos.GetRequest)Mockito.any())).
|
(ClientProtos.GetRequest)Mockito.any())).
|
||||||
thenThrow(new ServiceException(new RegionServerStoppedException("From Mockito")));
|
thenThrow(new ServiceException(new java.net.ConnectException("Connection refused")));
|
||||||
} catch (ServiceException e) {
|
} catch (ServiceException e) {
|
||||||
throw new IOException(e);
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
|
@ -355,6 +390,16 @@ public class TestClientNoCluster extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Override to check we are setting rpc timeout right.
|
||||||
|
*/
|
||||||
|
static class RpcTimeoutAsyncConnection extends AsyncConnectionImpl {
|
||||||
|
RpcTimeoutAsyncConnection(Configuration configuration, ConnectionRegistry registry,
|
||||||
|
String clusterId, User user) {
|
||||||
|
super(configuration, registry, clusterId, user);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fake many regionservers and many regions on a connection implementation.
|
* Fake many regionservers and many regions on a connection implementation.
|
||||||
*/
|
*/
|
||||||
|
|
Loading…
Reference in New Issue