});
MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
DistributedFileSystem fileSys = (DistributedFileSystem)cluster.getFileSystem();
DFSClient dfsClient = fileSys.getClient();
DFSQuorumReadMetrics metrics = dfsClient.quorumReadMetrics;
try {
Path file1 = new Path("quorumReadMaxOut.dat");
writeFile(fileSys, file1);
// time the pReadFile test
long t0, t1;
/*
* Basic test. Reads complete within timeout.
* Assert that there were no quorum reads.
*
*/
t0 = System.currentTimeMillis();
pReadFile(fileSys, file1);
t1 = System.currentTimeMillis();
long pReadTestTime = t1 - t0;
// assert that there were no quorum reads. 60ms + delta < 100ms
assertTrue(metrics.getParallelReadOps() == 0);
assertTrue(metrics.getParallelReadOpsInCurThread() == 0);
// set the timeout to 50ms;
/*
* Reads take longer than timeout. But, only one thread reading.
*
* Assert that there were quorum reads. But, none of the
* reads had to run in the current thread.
*/
dfsClient.setQuorumReadTimeout(50); // 50ms
t0 = System.currentTimeMillis();
pReadFile(fileSys, file1);
t1 = System.currentTimeMillis();
// assert that there were quorum reads
long pReadTestTimeNew = t1 - t0;
// assert that there were no quorum reads. 60ms + delta < 100ms
assertTrue(metrics.getParallelReadOps() > 0);
assertTrue(metrics.getParallelReadOpsInCurThread() == 0);
/*
* Multiple threads reading. Reads take longer than timeout.
*
* Assert that there were quorum reads. And that
* reads had to run in the current thread.
*/
int factor = 10;
int numParallelReads = numQuorumPoolThreads * factor;
long initialReadOpsValue = metrics.getParallelReadOps();
ExecutorService executor = Executors.newFixedThreadPool(numParallelReads);
ArrayList<Future<Void>> futures = new ArrayList<Future<Void>>();
for (int i = 0; i < numParallelReads; i++) {
futures.add(executor.submit(getPReadFileCallable(fileSys, file1)));
}
for (int i = 0; i < numParallelReads; i++) {
futures.get(i).get();
}
assertTrue(metrics.getParallelReadOps() > initialReadOpsValue);
assertTrue(metrics.getParallelReadOpsInCurThread() > 0);
cleanupFile(fileSys, file1);
} finally {
fileSys.close();
cluster.shutdown();