|
@@ -21,6 +21,7 @@ import org.apache.commons.lang3.RandomUtils;
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
+import org.apache.hadoop.ozone.chaos.TestProbability;
|
|
|
import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
|
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
|
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
|
@@ -28,6 +29,7 @@ import org.apache.hadoop.util.Time;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
@@ -37,7 +39,10 @@ import java.util.concurrent.ThreadPoolExecutor;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.ArrayBlockingQueue;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
/**
|
|
|
* A Simple Load generator for testing.
|
|
@@ -47,6 +52,8 @@ public class MiniOzoneLoadGenerator {
|
|
|
static final Logger LOG =
|
|
|
LoggerFactory.getLogger(MiniOzoneLoadGenerator.class);
|
|
|
|
|
|
+ private static String keyNameDelimiter = "_";
|
|
|
+
|
|
|
private ThreadPoolExecutor writeExecutor;
|
|
|
private int numWriteThreads;
|
|
|
// number of buffer to be allocated, each is allocated with length which
|
|
@@ -58,7 +65,13 @@ public class MiniOzoneLoadGenerator {
|
|
|
|
|
|
private final List<OzoneBucket> ozoneBuckets;
|
|
|
|
|
|
- MiniOzoneLoadGenerator(List<OzoneBucket> bucket, int numThreads,
|
|
|
+ private final AtomicInteger agedFileWrittenIndex;
|
|
|
+ private final ExecutorService agedFileExecutor;
|
|
|
+ private final OzoneBucket agedLoadBucket;
|
|
|
+ private final TestProbability agedWriteProbability;
|
|
|
+
|
|
|
+ MiniOzoneLoadGenerator(List<OzoneBucket> bucket,
|
|
|
+ OzoneBucket agedLoadBucket, int numThreads,
|
|
|
int numBuffers) {
|
|
|
this.ozoneBuckets = bucket;
|
|
|
this.numWriteThreads = numThreads;
|
|
@@ -68,6 +81,11 @@ public class MiniOzoneLoadGenerator {
|
|
|
new ThreadPoolExecutor.CallerRunsPolicy());
|
|
|
this.writeExecutor.prestartAllCoreThreads();
|
|
|
|
|
|
+ this.agedFileWrittenIndex = new AtomicInteger(0);
|
|
|
+ this.agedFileExecutor = Executors.newSingleThreadExecutor();
|
|
|
+ this.agedLoadBucket = agedLoadBucket;
|
|
|
+ this.agedWriteProbability = TestProbability.valueOf(10);
|
|
|
+
|
|
|
this.isWriteThreadRunning = new AtomicBoolean(false);
|
|
|
|
|
|
// allocate buffers and populate random data.
|
|
@@ -89,51 +107,111 @@ public class MiniOzoneLoadGenerator {
|
|
|
|
|
|
while (isWriteThreadRunning.get() &&
|
|
|
(Time.monotonicNow() < startTime + runTimeMillis)) {
|
|
|
- // choose a random buffer.
|
|
|
- int index = RandomUtils.nextInt();
|
|
|
- ByteBuffer buffer = buffers.get(index % numBuffers);
|
|
|
- int bufferCapacity = buffer.capacity();
|
|
|
-
|
|
|
- String keyName = threadName + "-" + index;
|
|
|
OzoneBucket bucket =
|
|
|
ozoneBuckets.get((int) (Math.random() * ozoneBuckets.size()));
|
|
|
- try (OzoneOutputStream stream = bucket.createKey(keyName,
|
|
|
- bufferCapacity, ReplicationType.RATIS, ReplicationFactor.THREE,
|
|
|
- new HashMap<>())) {
|
|
|
- stream.write(buffer.array());
|
|
|
+ try {
|
|
|
+ int index = RandomUtils.nextInt();
|
|
|
+ String keyName = writeData(index, bucket, threadName);
|
|
|
+
|
|
|
+ readData(bucket, keyName);
|
|
|
+
|
|
|
+ deleteKey(bucket, keyName);
|
|
|
} catch (Exception e) {
|
|
|
- LOG.error("LOADGEN: Create key:{} failed with exception, skipping",
|
|
|
- keyName, e);
|
|
|
- continue;
|
|
|
- // TODO: HDDS-1403.A key write can fail after multiple block writes
|
|
|
- // to closed container. add a break here once that is fixed.
|
|
|
+ LOG.error("LOADGEN: Exiting due to exception", e);
|
|
|
+ break;
|
|
|
}
|
|
|
+ }
|
|
|
+ // This will terminate other threads too.
|
|
|
+ isWriteThreadRunning.set(false);
|
|
|
+ LOG.info("Terminating IO thread:{}.", threadID);
|
|
|
+ }
|
|
|
|
|
|
- try (OzoneInputStream stream = bucket.readKey(keyName)) {
|
|
|
- byte[] readBuffer = new byte[bufferCapacity];
|
|
|
- int readLen = stream.read(readBuffer);
|
|
|
|
|
|
- if (readLen < bufferCapacity) {
|
|
|
- LOG.error("LOADGEN: Read mismatch, key:{} read data length:{} is " +
|
|
|
- "smaller than excepted:{}", keyName, readLen, bufferCapacity);
|
|
|
- break;
|
|
|
- }
|
|
|
+ private String writeData(int keyIndex, OzoneBucket bucket, String threadName)
|
|
|
+ throws Exception {
|
|
|
+ // choose a random buffer.
|
|
|
+ ByteBuffer buffer = buffers.get(keyIndex % numBuffers);
|
|
|
+ int bufferCapacity = buffer.capacity();
|
|
|
|
|
|
- if (!Arrays.equals(readBuffer, buffer.array())) {
|
|
|
- LOG.error("LOADGEN: Read mismatch, key:{} Read data does not match " +
|
|
|
- "the written data", keyName);
|
|
|
- break;
|
|
|
- }
|
|
|
+ String keyName = threadName + keyNameDelimiter + keyIndex;
|
|
|
+ try (OzoneOutputStream stream = bucket.createKey(keyName,
|
|
|
+ bufferCapacity, ReplicationType.RATIS, ReplicationFactor.THREE,
|
|
|
+ new HashMap<>())) {
|
|
|
+ stream.write(buffer.array());
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("LOADGEN: Create key:{} failed with exception, skipping",
|
|
|
+ keyName, t);
|
|
|
+ throw t;
|
|
|
+ }
|
|
|
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("LOADGEN: Read key:{} failed with exception", keyName, e);
|
|
|
- break;
|
|
|
+ return keyName;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void readData(OzoneBucket bucket, String keyName) throws Exception {
|
|
|
+ int index = Integer.valueOf(keyName.split(keyNameDelimiter)[1]);
|
|
|
+
|
|
|
+
|
|
|
+ ByteBuffer buffer = buffers.get(index % numBuffers);
|
|
|
+ int bufferCapacity = buffer.capacity();
|
|
|
+
|
|
|
+ try (OzoneInputStream stream = bucket.readKey(keyName)) {
|
|
|
+ byte[] readBuffer = new byte[bufferCapacity];
|
|
|
+ int readLen = stream.read(readBuffer);
|
|
|
+
|
|
|
+ if (readLen < bufferCapacity) {
|
|
|
+ throw new IOException("Read mismatch, key:" + keyName +
|
|
|
+ " read data length:" + readLen +
|
|
|
+ " is smaller than excepted:" + bufferCapacity);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!Arrays.equals(readBuffer, buffer.array())) {
|
|
|
+ throw new IOException("Read mismatch, key:" + keyName +
|
|
|
+ " read data does not match the written data");
|
|
|
}
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("LOADGEN: Read key:{} failed with exception", keyName, t);
|
|
|
+ throw t;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void deleteKey(OzoneBucket bucket, String keyName) throws Exception {
|
|
|
+ try {
|
|
|
+ bucket.deleteKey(keyName);
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("LOADGEN: Unable to delete key:{}", keyName, t);
|
|
|
+ throw t;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private String getKeyToRead() {
|
|
|
+ int currentIndex = agedFileWrittenIndex.get();
|
|
|
+ return currentIndex != 0 ?
|
|
|
+ String.valueOf(RandomUtils.nextInt(0, currentIndex)): null;
|
|
|
+ }
|
|
|
|
|
|
+ private void startAgedFilesLoad(long runTimeMillis) {
|
|
|
+ long threadID = Thread.currentThread().getId();
|
|
|
+ LOG.info("AGED LOADGEN: Started Aged IO Thread:{}.", threadID);
|
|
|
+ String threadName = Thread.currentThread().getName();
|
|
|
+ long startTime = Time.monotonicNow();
|
|
|
+
|
|
|
+ while (isWriteThreadRunning.get() &&
|
|
|
+ (Time.monotonicNow() < startTime + runTimeMillis)) {
|
|
|
+
|
|
|
+ String keyName = null;
|
|
|
try {
|
|
|
- bucket.deleteKey(keyName);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("LOADGEN: Unable to delete key:{}", keyName, e);
|
|
|
+ if (agedWriteProbability.isTrue()) {
|
|
|
+ keyName = writeData(agedFileWrittenIndex.incrementAndGet(),
|
|
|
+ agedLoadBucket, threadName);
|
|
|
+ } else {
|
|
|
+ keyName = getKeyToRead();
|
|
|
+ if (keyName != null) {
|
|
|
+ readData(agedLoadBucket, keyName);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("AGED LOADGEN: {} Exiting due to exception", keyName, t);
|
|
|
+ break;
|
|
|
}
|
|
|
}
|
|
|
// This will terminate other threads too.
|
|
@@ -141,7 +219,7 @@ public class MiniOzoneLoadGenerator {
|
|
|
LOG.info("Terminating IO thread:{}.", threadID);
|
|
|
}
|
|
|
|
|
|
- public void startIO(long time, TimeUnit timeUnit) {
|
|
|
+ void startIO(long time, TimeUnit timeUnit) {
|
|
|
List<CompletableFuture<Void>> writeFutures = new ArrayList<>();
|
|
|
LOG.info("Starting MiniOzoneLoadGenerator for time {}:{} with {} buffers " +
|
|
|
"and {} threads", time, timeUnit, numBuffers, numWriteThreads);
|
|
@@ -153,6 +231,9 @@ public class MiniOzoneLoadGenerator {
|
|
|
writeExecutor));
|
|
|
}
|
|
|
|
|
|
+ writeFutures.add(CompletableFuture.runAsync(() ->
|
|
|
+ startAgedFilesLoad(timeUnit.toMillis(time)), agedFileExecutor));
|
|
|
+
|
|
|
// Wait for IO to complete
|
|
|
for (CompletableFuture<Void> f : writeFutures) {
|
|
|
try {
|