HDDS-1532. Improve the concurrent testing framework of Freon. (#957)
This commit is contained in:
parent
22b36dd311
commit
3d020e914f
|
@ -64,7 +64,6 @@ import com.fasterxml.jackson.annotation.PropertyAccessor;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import static java.lang.Math.min;
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
import org.apache.commons.lang3.time.DurationFormatUtils;
|
import org.apache.commons.lang3.time.DurationFormatUtils;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -102,6 +101,8 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
|
|
||||||
private static final int QUANTILES = 10;
|
private static final int QUANTILES = 10;
|
||||||
|
|
||||||
|
private static final int CHECK_INTERVAL_MILLIS = 5000;
|
||||||
|
|
||||||
private byte[] keyValueBuffer = null;
|
private byte[] keyValueBuffer = null;
|
||||||
|
|
||||||
private static final String DIGEST_ALGORITHM = "MD5";
|
private static final String DIGEST_ALGORITHM = "MD5";
|
||||||
|
@ -180,7 +181,7 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
|
|
||||||
private OzoneClient ozoneClient;
|
private OzoneClient ozoneClient;
|
||||||
private ObjectStore objectStore;
|
private ObjectStore objectStore;
|
||||||
private ExecutorService processor;
|
private ExecutorService executor;
|
||||||
|
|
||||||
private long startTime;
|
private long startTime;
|
||||||
private long jobStartTime;
|
private long jobStartTime;
|
||||||
|
@ -259,9 +260,8 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Number of Threads: " + numOfThreads);
|
LOG.info("Number of Threads: " + numOfThreads);
|
||||||
threadPoolSize =
|
threadPoolSize = numOfThreads;
|
||||||
min(numOfVolumes, numOfThreads);
|
executor = Executors.newFixedThreadPool(threadPoolSize);
|
||||||
processor = Executors.newFixedThreadPool(threadPoolSize);
|
|
||||||
addShutdownHook();
|
addShutdownHook();
|
||||||
|
|
||||||
LOG.info("Number of Volumes: {}.", numOfVolumes);
|
LOG.info("Number of Volumes: {}.", numOfVolumes);
|
||||||
|
@ -270,9 +270,8 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
LOG.info("Key size: {} bytes", keySize);
|
LOG.info("Key size: {} bytes", keySize);
|
||||||
LOG.info("Buffer size: {} bytes", bufferSize);
|
LOG.info("Buffer size: {} bytes", bufferSize);
|
||||||
for (int i = 0; i < numOfVolumes; i++) {
|
for (int i = 0; i < numOfVolumes; i++) {
|
||||||
String volume = "vol-" + i + "-" +
|
String volumeName = "vol-" + i + "-" + RandomStringUtils.randomNumeric(5);
|
||||||
RandomStringUtils.randomNumeric(5);
|
executor.submit(new VolumeProcessor(volumeName));
|
||||||
processor.submit(new OfflineProcessor(volume));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Thread validator = null;
|
Thread validator = null;
|
||||||
|
@ -301,8 +300,17 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
|
|
||||||
progressbar.start();
|
progressbar.start();
|
||||||
|
|
||||||
processor.shutdown();
|
// wait until all keys are added or exception occurred.
|
||||||
processor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
|
while ((numberOfKeysAdded.get() != numOfVolumes * numOfBuckets * numOfKeys)
|
||||||
|
&& exception == null) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(CHECK_INTERVAL_MILLIS);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
executor.shutdown();
|
||||||
|
executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
|
||||||
completed = true;
|
completed = true;
|
||||||
|
|
||||||
if (exception != null) {
|
if (exception != null) {
|
||||||
|
@ -571,15 +579,10 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private class OfflineProcessor implements Runnable {
|
private class VolumeProcessor implements Runnable {
|
||||||
|
|
||||||
private int totalBuckets;
|
|
||||||
private int totalKeys;
|
|
||||||
private String volumeName;
|
private String volumeName;
|
||||||
|
|
||||||
OfflineProcessor(String volumeName) {
|
VolumeProcessor(String volumeName) {
|
||||||
this.totalBuckets = numOfBuckets;
|
|
||||||
this.totalKeys = numOfKeys;
|
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -604,14 +607,31 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
Long threadKeyWriteTime = 0L;
|
for (int i = 0; i < numOfBuckets; i++) {
|
||||||
for (int j = 0; j < totalBuckets; j++) {
|
String bucketName = "bucket-" + i + "-" +
|
||||||
String bucketName = "bucket-" + j + "-" +
|
|
||||||
RandomStringUtils.randomNumeric(5);
|
RandomStringUtils.randomNumeric(5);
|
||||||
try {
|
BucketProcessor bp = new BucketProcessor(volume, bucketName);
|
||||||
|
executor.submit(bp);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class BucketProcessor implements Runnable {
|
||||||
|
private OzoneVolume volume;
|
||||||
|
private String bucketName;
|
||||||
|
|
||||||
|
BucketProcessor(OzoneVolume volume, String bucketName) {
|
||||||
|
this.volume = volume;
|
||||||
|
this.bucketName = bucketName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@SuppressFBWarnings("REC_CATCH_EXCEPTION")
|
||||||
|
public void run() {
|
||||||
LOG.trace("Creating bucket: {} in volume: {}",
|
LOG.trace("Creating bucket: {} in volume: {}",
|
||||||
bucketName, volume.getName());
|
bucketName, volume.getName());
|
||||||
start = System.nanoTime();
|
long start = System.nanoTime();
|
||||||
|
OzoneBucket bucket;
|
||||||
try (Scope scope = GlobalTracer.get().buildSpan("createBucket")
|
try (Scope scope = GlobalTracer.get().buildSpan("createBucket")
|
||||||
.startActive(true)) {
|
.startActive(true)) {
|
||||||
volume.createBucket(bucketName);
|
volume.createBucket(bucketName);
|
||||||
|
@ -620,29 +640,52 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
.update(bucketCreationDuration);
|
.update(bucketCreationDuration);
|
||||||
bucketCreationTime.getAndAdd(bucketCreationDuration);
|
bucketCreationTime.getAndAdd(bucketCreationDuration);
|
||||||
numberOfBucketsCreated.getAndIncrement();
|
numberOfBucketsCreated.getAndIncrement();
|
||||||
|
|
||||||
|
bucket = volume.getBucket(bucketName);
|
||||||
|
} catch (IOException e) {
|
||||||
|
exception = e;
|
||||||
|
LOG.error("Could not create bucket ", e);
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
|
||||||
for (int k = 0; k < totalKeys; k++) {
|
for (int i = 0; i < numOfKeys; i++) {
|
||||||
String key = "key-" + k + "-" +
|
String keyName = "key-" + i + "-" + RandomStringUtils.randomNumeric(5);
|
||||||
RandomStringUtils.randomNumeric(5);
|
KeyProcessor kp = new KeyProcessor(bucket, keyName);
|
||||||
byte[] randomValue =
|
executor.submit(kp);
|
||||||
DFSUtil.string2Bytes(UUID.randomUUID().toString());
|
}
|
||||||
try {
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class KeyProcessor implements Runnable {
|
||||||
|
private OzoneBucket bucket;
|
||||||
|
private String keyName;
|
||||||
|
|
||||||
|
KeyProcessor(OzoneBucket bucket, String keyName) {
|
||||||
|
this.bucket = bucket;
|
||||||
|
this.keyName = keyName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@SuppressFBWarnings("REC_CATCH_EXCEPTION")
|
||||||
|
public void run() {
|
||||||
|
String bucketName = bucket.getName();
|
||||||
|
String volumeName = bucket.getVolumeName();
|
||||||
LOG.trace("Adding key: {} in bucket: {} of volume: {}",
|
LOG.trace("Adding key: {} in bucket: {} of volume: {}",
|
||||||
key, bucket, volume);
|
keyName, bucketName, volumeName);
|
||||||
|
byte[] randomValue = DFSUtil.string2Bytes(UUID.randomUUID().toString());
|
||||||
|
try {
|
||||||
long keyCreateStart = System.nanoTime();
|
long keyCreateStart = System.nanoTime();
|
||||||
try (Scope scope = GlobalTracer.get().buildSpan("createKey")
|
try (Scope scope = GlobalTracer.get().buildSpan("createKey")
|
||||||
.startActive(true)) {
|
.startActive(true)) {
|
||||||
OzoneOutputStream os =
|
OzoneOutputStream os = bucket.createKey(keyName, keySize, type,
|
||||||
bucket
|
factor, new HashMap<>());
|
||||||
.createKey(key, keySize, type, factor, new HashMap<>());
|
|
||||||
long keyCreationDuration = System.nanoTime() - keyCreateStart;
|
long keyCreationDuration = System.nanoTime() - keyCreateStart;
|
||||||
histograms.get(FreonOps.KEY_CREATE.ordinal())
|
histograms.get(FreonOps.KEY_CREATE.ordinal())
|
||||||
.update(keyCreationDuration);
|
.update(keyCreationDuration);
|
||||||
keyCreationTime.getAndAdd(keyCreationDuration);
|
keyCreationTime.getAndAdd(keyCreationDuration);
|
||||||
|
|
||||||
long keyWriteStart = System.nanoTime();
|
long keyWriteStart = System.nanoTime();
|
||||||
try (Scope writeScope = GlobalTracer.get()
|
try (Scope writeScope = GlobalTracer.get().buildSpan("writeKeyData")
|
||||||
.buildSpan("writeKeyData")
|
|
||||||
.startActive(true)) {
|
.startActive(true)) {
|
||||||
for (long nrRemaining = keySize - randomValue.length;
|
for (long nrRemaining = keySize - randomValue.length;
|
||||||
nrRemaining > 0; nrRemaining -= bufferSize) {
|
nrRemaining > 0; nrRemaining -= bufferSize) {
|
||||||
|
@ -651,41 +694,31 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
}
|
}
|
||||||
os.write(randomValue);
|
os.write(randomValue);
|
||||||
os.close();
|
os.close();
|
||||||
}
|
|
||||||
|
|
||||||
long keyWriteDuration = System.nanoTime() - keyWriteStart;
|
long keyWriteDuration = System.nanoTime() - keyWriteStart;
|
||||||
|
|
||||||
threadKeyWriteTime += keyWriteDuration;
|
|
||||||
histograms.get(FreonOps.KEY_WRITE.ordinal())
|
histograms.get(FreonOps.KEY_WRITE.ordinal())
|
||||||
.update(keyWriteDuration);
|
.update(keyWriteDuration);
|
||||||
|
keyWriteTime.getAndAdd(keyWriteDuration);
|
||||||
totalBytesWritten.getAndAdd(keySize);
|
totalBytesWritten.getAndAdd(keySize);
|
||||||
numberOfKeysAdded.getAndIncrement();
|
numberOfKeysAdded.getAndIncrement();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (validateWrites) {
|
if (validateWrites) {
|
||||||
MessageDigest tmpMD = (MessageDigest)commonInitialMD.clone();
|
MessageDigest tmpMD = (MessageDigest)commonInitialMD.clone();
|
||||||
tmpMD.update(randomValue);
|
tmpMD.update(randomValue);
|
||||||
boolean validate = validationQueue.offer(
|
boolean validate = validationQueue.offer(
|
||||||
new KeyValidate(bucket, key, tmpMD.digest()));
|
new KeyValidate(bucket, keyName, tmpMD.digest()));
|
||||||
if (validate) {
|
if (validate) {
|
||||||
LOG.trace("Key {}, is queued for validation.", key);
|
LOG.trace("Key {}, is queued for validation.", keyName);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
exception = e;
|
exception = e;
|
||||||
LOG.error("Exception while adding key: {} in bucket: {}" +
|
LOG.error("Exception while adding key: {} in bucket: {}" +
|
||||||
" of volume: {}.", key, bucket, volume, e);
|
" of volume: {}.", keyName, bucketName, volumeName, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
|
||||||
exception = e;
|
|
||||||
LOG.error("Exception while creating bucket: {}" +
|
|
||||||
" in volume: {}.", bucketName, volume, e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
keyWriteTime.getAndAdd(threadKeyWriteTime);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private final class FreonJobInfo {
|
private final class FreonJobInfo {
|
||||||
|
@ -1028,4 +1061,9 @@ public final class RandomKeyGenerator implements Callable<Void> {
|
||||||
public void setValidateWrites(boolean validateWrites) {
|
public void setValidateWrites(boolean validateWrites) {
|
||||||
this.validateWrites = validateWrites;
|
this.validateWrites = validateWrites;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public int getThreadPoolSize() {
|
||||||
|
return threadPoolSize;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -127,4 +127,19 @@ public class TestRandomKeyGenerator {
|
||||||
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
|
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
|
||||||
Assert.assertEquals(1, randomKeyGenerator.getSuccessfulValidationCount());
|
Assert.assertEquals(1, randomKeyGenerator.getSuccessfulValidationCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testThreadPoolSize() throws Exception {
|
||||||
|
RandomKeyGenerator randomKeyGenerator =
|
||||||
|
new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
|
||||||
|
randomKeyGenerator.setNumOfVolumes(1);
|
||||||
|
randomKeyGenerator.setNumOfBuckets(1);
|
||||||
|
randomKeyGenerator.setNumOfKeys(1);
|
||||||
|
randomKeyGenerator.setFactor(ReplicationFactor.THREE);
|
||||||
|
randomKeyGenerator.setType(ReplicationType.RATIS);
|
||||||
|
randomKeyGenerator.setNumOfThreads(10);
|
||||||
|
randomKeyGenerator.call();
|
||||||
|
Assert.assertEquals(10, randomKeyGenerator.getThreadPoolSize());
|
||||||
|
Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue