System.exit(-1);
}
}, timeout);
}
BookKeeper bkc = null;
try {
int numFiles = Integer.valueOf(cmd.getOptionValue("numconcurrent", "1"));
int numThreads = Math.min(numFiles, 1000);
byte[] data = sb.toString().getBytes();
long runid = System.currentTimeMillis();
List<Callable<Long>> clients = new ArrayList<Callable<Long>>();
if (target.equals("bk")) {
String zkservers = cmd.getOptionValue("zkservers", "localhost:2181");
int bkensemble = Integer.valueOf(cmd.getOptionValue("bkensemble", "3"));
int bkquorum = Integer.valueOf(cmd.getOptionValue("bkquorum", "2"));
int bkthrottle = Integer.valueOf(cmd.getOptionValue("bkthrottle", "10000"));
ClientConfiguration conf = new ClientConfiguration();
conf.setThrottleValue(bkthrottle);
conf.setZkServers(zkservers);
bkc = new BookKeeper(conf);
List<LedgerHandle> handles = new ArrayList<LedgerHandle>();
for (int i = 0; i < numFiles; i++) {
handles.add(bkc.createLedger(bkensemble, bkquorum, DigestType.CRC32, new byte[] {'a', 'b'}));
}
for (int i = 0; i < numFiles; i++) {
clients.add(new BKClient(handles, data, runfor, cmd.hasOption("sync")));
}
} else if (target.equals("hdfs")) {
FileSystem fs = FileSystem.get(new Configuration());
LOG.info("Default replication for HDFS: {}", fs.getDefaultReplication());
List<FSDataOutputStream> streams = new ArrayList<FSDataOutputStream>();
for (int i = 0; i < numFiles; i++) {
String path = cmd.getOptionValue("path", "/foobar");
streams.add(fs.create(new Path(path + runid + "_" + i)));
}
for (int i = 0; i < numThreads; i++) {
clients.add(new HDFSClient(streams, data, runfor));
}
} else if (target.equals("fs")) {
List<FileOutputStream> streams = new ArrayList<FileOutputStream>();
for (int i = 0; i < numFiles; i++) {
String path = cmd.getOptionValue("path", "/foobar " + i);
streams.add(new FileOutputStream(path + runid + "_" + i));
}
for (int i = 0; i < numThreads; i++) {
clients.add(new FileClient(streams, data, runfor));
}
} else {
LOG.error("Unknown option: " + target);
throw new IllegalArgumentException("Unknown target " + target);
}
ExecutorService executor = Executors.newFixedThreadPool(numThreads);
long start = System.currentTimeMillis();
List<Future<Long>> results = executor.invokeAll(clients,
10, TimeUnit.MINUTES);
long end = System.currentTimeMillis();
long count = 0;
for (Future<Long> r : results) {
if (!r.isDone()) {
LOG.warn("Job didn't complete");
System.exit(2);
}
long c = r.get();
if (c == 0) {
LOG.warn("Task didn't complete");
}
count += c;
}
long time = end-start;
LOG.info("Finished processing writes (ms): {} TPT: {} op/s",
time, count/((double)time/1000));
executor.shutdown();
} catch (ExecutionException ee) {
LOG.error("Exception in worker", ee);
} catch (KeeperException ke) {
LOG.error("Error accessing zookeeper", ke);
} catch (BKException e) {
LOG.error("Error accessing bookkeeper", e);
} catch (IOException ioe) {
LOG.error("I/O exception during benchmark", ioe);
} catch (InterruptedException ie) {
LOG.error("Benchmark interrupted", ie);
} finally {
if (bkc != null) {
try {
bkc.close();
} catch (BKException bke) {
LOG.error("Error closing bookkeeper client", bke);
} catch (InterruptedException ie) {
LOG.warn("Interrupted closing bookkeeper client", ie);
}