more options on stress rolling restart test, variable number of fields, variable length of text

This commit is contained in:
kimchy 2010-10-16 03:26:11 +02:00
parent b09675dcfa
commit 1587a3815e
1 changed files with 52 additions and 13 deletions

View File

@ -21,12 +21,15 @@ package org.elasticsearch.test.stress.rollingrestart;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.common.UUID;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.jsr166y.ThreadLocalRandom;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.node.Node;
@ -34,7 +37,6 @@ import org.elasticsearch.node.NodeBuilder;
import org.elasticsearch.node.internal.InternalNode;
import java.io.File;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import static org.elasticsearch.index.query.xcontent.QueryBuilders.*;
@ -48,6 +50,8 @@ public class RollingRestartStressTest {
private int numberOfNodes = 4;
private int textTokens = 150;
private int numberOfFields = 10;
private long initialNumberOfDocs = 100000;
private int indexers = 0;
@ -64,6 +68,7 @@ public class RollingRestartStressTest {
private AtomicLong indexCounter = new AtomicLong();
public RollingRestartStressTest numberOfNodes(int numberOfNodes) {
this.numberOfNodes = numberOfNodes;
return this;
@ -74,6 +79,16 @@ public class RollingRestartStressTest {
return this;
}
public RollingRestartStressTest textTokens(int textTokens) {
this.textTokens = textTokens;
return this;
}
public RollingRestartStressTest numberOfFields(int numberOfFields) {
this.numberOfFields = numberOfFields;
return this;
}
public RollingRestartStressTest indexers(int indexers) {
this.indexers = indexers;
return this;
@ -152,11 +167,17 @@ public class RollingRestartStressTest {
}
for (int i = 0; i < indexerThreads.length; i++) {
indexerThreads[i].closed = true;
indexerThreads[i].close = true;
}
Thread.sleep(indexerThrottle.millis() + 10000);
for (int i = 0; i < indexerThreads.length; i++) {
if (!indexerThreads[i].closed) {
logger.warn("thread not closed!");
}
}
client.client().admin().indices().prepareRefresh().execute().actionGet();
// check the count
@ -176,32 +197,48 @@ public class RollingRestartStressTest {
private class Indexer extends Thread {
volatile boolean close = false;
volatile boolean closed = false;
@Override public void run() {
while (true) {
if (closed) {
if (close) {
closed = true;
return;
}
try {
indexDoc();
Thread.sleep(indexerThrottle.millis());
} catch (IOException e) {
logger.warn("failed to index", e);
} catch (InterruptedException e) {
break;
} catch (Exception e) {
logger.warn("failed to index / sleep", e);
}
}
}
}
private void indexDoc() throws IOException {
long id = indexCounter.incrementAndGet();
private void indexDoc() throws Exception {
StringBuffer sb = new StringBuffer();
XContentBuilder json = XContentFactory.jsonBuilder().startObject()
.field("field", "value" + ThreadLocalRandom.current().nextInt());
int fields = ThreadLocalRandom.current().nextInt() % numberOfFields;
for (int i = 0; i < fields; i++) {
json.field("num_" + i, ThreadLocalRandom.current().nextDouble());
int tokens = ThreadLocalRandom.current().nextInt() % textTokens;
sb.setLength(0);
for (int j = 0; j < tokens; j++) {
sb.append(UUID.randomBase64UUID()).append(' ');
}
json.field("text_" + i, sb.toString());
}
json.endObject();
client.client().prepareIndex("test", "type1")
.setSource(XContentFactory.jsonBuilder().startObject()
.field("field", "value" + id)
.endObject())
.setSource(json)
.execute().actionGet();
indexCounter.incrementAndGet();
}
public static void main(String[] args) throws Exception {
@ -214,9 +251,11 @@ public class RollingRestartStressTest {
.settings(settings)
.numberOfNodes(4)
.initialNumberOfDocs(100000)
.textTokens(150)
.numberOfFields(10)
.cleanNodeWork(true)
.indexers(5)
.indexerThrottle(TimeValue.timeValueMillis(100))
.indexerThrottle(TimeValue.timeValueMillis(50))
.period(TimeValue.timeValueMinutes(10));
test.run();