HBASE-16122 PerformanceEvaluation should provide user friendly hint when client threads argument is missing (Konstantin Ryakhovskiy)

This commit is contained in:
tedyu 2016-06-28 17:42:51 -07:00
parent 8bc4d4131c
commit 394e7224a9
2 changed files with 51 additions and 1 deletions

View File

@ -36,6 +36,7 @@ import java.util.Map;
import java.util.Queue;
import java.util.Random;
import java.util.TreeMap;
import java.util.NoSuchElementException;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@ -2139,7 +2140,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
if (isCommandClass(cmd)) {
opts.cmdName = cmd;
opts.numClientThreads = Integer.parseInt(args.remove());
try {
opts.numClientThreads = Integer.parseInt(args.remove());
} catch (NoSuchElementException | NumberFormatException e) {
throw new IllegalArgumentException("Command " + cmd + " does not have threads number", e);
}
if (opts.size != DEFAULT_OPTS.size &&
opts.perClientRunRows != DEFAULT_OPTS.perClientRunRows) {
throw new IllegalArgumentException(rows + " and " + size +

View File

@ -25,6 +25,9 @@ import java.io.IOException;
import java.io.InputStreamReader;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.NoSuchElementException;
import java.util.Queue;
import java.util.LinkedList;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -138,4 +141,46 @@ public class TestPerformanceEvaluation {
double median = snapshot.getMedian();
assertTrue(median != 0 && median != 1 && median != valueSize);
}
@Test
public void testParseOptsWithThreads() {
Queue<String> opts = new LinkedList<>();
String cmdName = "sequentialWrite";
int threads = 1;
opts.offer(cmdName);
opts.offer(String.valueOf(threads));
PerformanceEvaluation.TestOptions options = PerformanceEvaluation.parseOpts(opts);
assertNotNull(options);
assertNotNull(options.getCmdName());
assertEquals(cmdName, options.getCmdName());
assertEquals(threads, options.getNumClientThreads());
}
@Test
public void testParseOptsWrongThreads() {
Queue<String> opts = new LinkedList<>();
String cmdName = "sequentialWrite";
opts.offer(cmdName);
opts.offer("qq");
try {
PerformanceEvaluation.parseOpts(opts);
} catch (IllegalArgumentException e) {
System.out.println(e.getMessage());
assertEquals("Command " + cmdName + " does not have threads number", e.getMessage());
assertTrue(e.getCause() instanceof NumberFormatException);
}
}
@Test
public void testParseOptsNoThreads() {
Queue<String> opts = new LinkedList<>();
String cmdName = "sequentialWrite";
try {
PerformanceEvaluation.parseOpts(opts);
} catch (IllegalArgumentException e) {
System.out.println(e.getMessage());
assertEquals("Command " + cmdName + " does not have threads number", e.getMessage());
assertTrue(e.getCause() instanceof NoSuchElementException);
}
}
}