HBASE-14758 Add UT case for unchecked error/exception thrown in AsyncProcess#sendMultiAction (Yu Li)

This commit is contained in:
tedyu 2015-11-04 07:20:59 -08:00
parent ceddec9141
commit cfc5348b5e
1 changed files with 40 additions and 0 deletions

View File

@ -29,7 +29,10 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
@ -1049,4 +1052,41 @@ public class TestAsyncProcess {
return p;
}
static class MyThreadPoolExecutor extends ThreadPoolExecutor {
public MyThreadPoolExecutor(int coreThreads, int maxThreads, long keepAliveTime,
TimeUnit timeunit, BlockingQueue<Runnable> blockingqueue) {
super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue);
}
@Override
public Future submit(Runnable runnable) {
throw new OutOfMemoryError("OutOfMemory error thrown by means");
}
}
static class AsyncProcessForThrowableCheck extends AsyncProcess {
public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf,
ExecutorService pool) {
super(hc, conf, pool, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(
conf));
}
}
@Test
public void testUncheckedException() throws Exception {
// Test the case pool.submit throws unchecked exception
ClusterConnection hc = createHConnection();
MyThreadPoolExecutor myPool =
new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(200));
AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, conf, myPool);
List<Put> puts = new ArrayList<Put>();
puts.add(createPut(1, true));
ap.submit(DUMMY_TABLE, puts, false, null, false);
Assert.assertTrue(puts.isEmpty());
}
}