long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER;
HTable table = new HTable(conf, TABLE_NAME);
// Create multi-threaded writer and start it. We write multiple columns/CFs and verify
// their integrity, therefore multi-put is necessary.
MultiThreadedWriter writer =
new MultiThreadedWriter(dataGen, conf, TABLE_NAME);
writer.setMultiPut(true);
LOG.info("Starting writer; the number of keys to write is " + keysToWrite);
// TODO : Need to see if tag support has to be given here in the integration test suite
writer.start(1, keysToWrite, WRITER_THREADS);
// Now, do scans.
long now = EnvironmentEdgeManager.currentTimeMillis();
long timeLimit = now + (maxRuntime * 60000);
boolean isWriterDone = false;
while (now < timeLimit && !isWriterDone) {
LOG.info("Starting the scan; wrote approximately "
+ dataGen.getTotalNumberOfKeys() + " keys");
isWriterDone = writer.isDone();
if (isWriterDone) {
LOG.info("Scanning full result, writer is done");
}
Scan scan = new Scan();
for (byte[] cf : dataGen.getColumnFamilies()) {
scan.addFamily(cf);
}
scan.setFilter(dataGen.getScanFilter());
scan.setLoadColumnFamiliesOnDemand(true);
// The number of keys we can expect from scan - lower bound (before scan).
// Not a strict lower bound - writer knows nothing about filters, so we report
// this from generator. Writer might have generated the value but not put it yet.
long onesGennedBeforeScan = dataGen.getExpectedNumberOfKeys();
long startTs = EnvironmentEdgeManager.currentTimeMillis();
ResultScanner results = table.getScanner(scan);
long resultCount = 0;
Result result = null;
// Verify and count the results.
while ((result = results.next()) != null) {
boolean isOk = writer.verifyResultAgainstDataGenerator(result, true, true);
Assert.assertTrue("Failed to verify [" + Bytes.toString(result.getRow())+ "]", isOk);
++resultCount;
}
long timeTaken = EnvironmentEdgeManager.currentTimeMillis() - startTs;
// Verify the result count.
long onesGennedAfterScan = dataGen.getExpectedNumberOfKeys();
Assert.assertTrue("Read " + resultCount + " keys when at most " + onesGennedAfterScan
+ " were generated ", onesGennedAfterScan >= resultCount);
if (isWriterDone) {
Assert.assertTrue("Read " + resultCount + " keys; the writer is done and "
+ onesGennedAfterScan + " keys were generated", onesGennedAfterScan == resultCount);
} else if (onesGennedBeforeScan * 0.9 > resultCount) {
LOG.warn("Read way too few keys (" + resultCount + "/" + onesGennedBeforeScan
+ ") - there might be a problem, or the writer might just be slow");
}
LOG.info("Scan took " + timeTaken + "ms");
if (!isWriterDone) {
Thread.sleep(WAIT_BETWEEN_SCANS_MS);
now = EnvironmentEdgeManager.currentTimeMillis();
}
}
Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures());
Assert.assertTrue("Writer is not done", isWriterDone);
// Assert.fail("Boom!");
}