Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Refactoring on index benchmark execution and ControlledExecutor #50

Merged
merged 15 commits into from
Feb 27, 2023
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
242 changes: 33 additions & 209 deletions src/main/java/org/apache/solr/benchmarks/BenchmarksMain.java
Original file line number Diff line number Diff line change
@@ -1,45 +1,12 @@
package org.apache.solr.benchmarks;

import java.io.BufferedReader;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.io.StringReader;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import java.util.zip.GZIPInputStream;

import org.apache.commons.io.IOUtils;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.entity.BasicHttpEntity;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.message.BasicHeader;
import org.apache.http.util.EntityUtils;
import org.apache.solr.benchmarks.beans.IndexBenchmark;
import org.apache.solr.benchmarks.beans.QueryBenchmark;
import org.apache.solr.benchmarks.readers.JsonlFileType;
import org.apache.solr.benchmarks.indexing.DocReader;
import org.apache.solr.benchmarks.indexing.FileDocReader;
import org.apache.solr.benchmarks.indexing.IndexBatchSupplier;
import org.apache.solr.benchmarks.solrcloud.SolrCloud;
import org.apache.solr.benchmarks.solrcloud.SolrNode;
import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
Expand All @@ -51,15 +18,19 @@
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.JsonRecordReader;
import org.apache.solr.common.util.NamedList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.*;
import java.lang.invoke.MethodHandles;
import java.text.ParseException;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.function.Supplier;
import java.util.zip.GZIPInputStream;

public class BenchmarksMain {

Expand Down Expand Up @@ -99,7 +70,9 @@ public static void runQueryBenchmarks(List<QueryBenchmark> queryBenchmarks, Stri
for (int threads = benchmark.minThreads; threads <= benchmark.maxThreads; threads++) {
QueryGenerator queryGenerator = new QueryGenerator(benchmark);
HttpSolrClient client = new HttpSolrClient.Builder(baseUrl).build();
ControlledExecutor controlledExecutor = new ControlledExecutor(threads,
ControlledExecutor controlledExecutor = new ControlledExecutor(
benchmark.name,
threads,
benchmark.durationSecs,
benchmark.rpm,
benchmark.totalCount,
Expand Down Expand Up @@ -174,17 +147,14 @@ public static void runIndexingBenchmarks(List<IndexBenchmark> indexBenchmarks, S
}
}

private static Supplier<Runnable> getQuerySupplier(QueryGenerator queryGenerator, HttpSolrClient client, String collection) {
private static Supplier<Callable> getQuerySupplier(QueryGenerator queryGenerator, HttpSolrClient client, String collection) {
return () -> {
QueryRequest qr = queryGenerator.nextRequest();
if (qr == null) return null;
return () -> {
try {
NamedList<Object> rsp = client.request(qr, collection);
printErrOutput(qr, rsp);
} catch (Exception e) {
log.error("Failed to execute request: " + qr, e);
}
NamedList<Object> rsp = client.request(qr, collection);
printErrOutput(qr, rsp);
return null;
};
};
}
Expand Down Expand Up @@ -244,189 +214,43 @@ static void indexTSV(String baseUrl, String collection, int threads, IndexBenchm
client.commit(collection);
client.close();
}

static void indexJsonSimple(String baseUrl, String collection, int threads, IndexBenchmark.Setup setup, IndexBenchmark benchmark) throws Exception {

long start = System.currentTimeMillis();

BufferedReader br = JsonlFileType.getBufferedReader(Util.resolveSuitePath(benchmark.datasetFile));
ConcurrentUpdateSolrClient client = new ConcurrentUpdateSolrClient.Builder(baseUrl).withThreadCount(threads).build();

String line;
int count = 0;
int errors = 0;

ObjectMapper mapper = new ObjectMapper();
while((line = br.readLine()) != null) {
count++;
if (count % 1_000_000 == 0) System.out.println("\tDocs read: "+count+", Errors: "+errors+", time: "+((System.currentTimeMillis() - start) / 1000));
if (count > benchmark.maxDocs) break;

SolrInputDocument doc = null;
try {
Map<String, Object> map = mapper.readValue(line, Map.class);
doc = new SolrInputDocument();
for (String key: map.keySet()) {
doc.addField(key, map.get(key));
}
client.add(collection, doc);
} catch (Exception ex) {
errors++;
}
}

client.blockUntilFinished();
client.commit(collection);
client.close();

br.close();

log.info("Indexed " + (count - errors) + " docs." + "time taken : " + ((System.currentTimeMillis() - start) / 1000));
}

static void indexJsonComplex(String baseUrl, String collection, int threads, IndexBenchmark.Setup setup, IndexBenchmark benchmark) throws Exception {

long start = System.currentTimeMillis();
CloseableHttpClient httpClient = HttpClientUtil.createClient(null);

final ExecutorService executor = Executors.newFixedThreadPool(threads);

long count;
AtomicInteger tasks = new AtomicInteger();
AtomicInteger completed = new AtomicInteger();

try {
HttpClusterStateProvider stateProvider = new HttpClusterStateProvider(Collections.singletonList(baseUrl), httpClient);
DocCollection coll = stateProvider.getCollection(collection);

DocRouter docRouter = coll.getRouter();
Map<String, String> shardVsLeader = new HashMap<>();

for (Slice slice : coll.getSlices()) {
Replica leader = slice.getLeader();
shardVsLeader.put(slice.getName(), leader.getBaseUrl() + "/" + leader.getCoreName() + "/update/json/docs");
}
JsonRecordReader rdr = JsonRecordReader.getInst("/", Collections.singletonList(benchmark.idField+":/"+benchmark.idField));
Map<String, List<String>> shardVsDocs = new HashMap<>();
File datasetFile = Util.resolveSuitePath(benchmark.datasetFile);
BufferedReader br = JsonlFileType.getBufferedReader(datasetFile);
count = 0;


String line;
String[] id = new String[1];
JsonRecordReader.Handler handler = (map, s) -> id[0] =
map.get(benchmark.idField) instanceof String?
(String) map.get(benchmark.idField):
map.get(benchmark.idField).toString();

RateLimiter rateLimiter = benchmark.rpm == null? null: new RateLimiter(benchmark.rpm);

try {
while ((line = br.readLine()) != null) {
line = line.trim();
if (line.isEmpty()) continue;
count++;

if (count < benchmark.offset) continue;

rdr.streamRecords(new StringReader(line), handler);
Slice targetSlice = docRouter.getTargetSlice(id[0], null, null, null, coll);
List<String> docs = shardVsDocs.get(targetSlice.getName());
if (docs == null) shardVsDocs.put(targetSlice.getName(), docs = new ArrayList<>(benchmark.batchSize));
if (count % 1_000_000 == 0)
log.info("\tDocs read: " + count + ", indexed: " + (completed.get() * benchmark.batchSize) + ", time: " + ((System.currentTimeMillis() - start) / 1000));
if (count > benchmark.maxDocs) break;
// _version_ must be removed or adding doc will fail
line = line.replaceAll("\"_version_\":\\d*,*", "");
docs.add(line);
if (docs.size() >= benchmark.batchSize) {
shardVsDocs.remove(targetSlice.getName());

if (rateLimiter != null) rateLimiter.waitIfRequired();
executor.submit(new UploadDocs(docs, httpClient,
shardVsLeader.get(targetSlice.getName()),
tasks, completed
));
}
try (DocReader docReader = new FileDocReader(datasetFile, benchmark.maxDocs != null ? benchmark.maxDocs.longValue() : null, benchmark.offset)) {
try (IndexBatchSupplier indexBatchSupplier = new IndexBatchSupplier(docReader, benchmark, coll, httpClient, shardVsLeader)) {
ControlledExecutor controlledExecutor = new ControlledExecutor(
benchmark.name,
threads,
benchmark.durationSecs,
benchmark.rpm,
null, //total is controlled by docReader's maxDocs
0,
indexBatchSupplier);
controlledExecutor.run();
HttpSolrClient client = new HttpSolrClient.Builder(baseUrl).build();
client.commit(collection);
client.close();

log.info("Indexed " + indexBatchSupplier.getDocsIndexed() + " docs." + "time taken : " + ((System.currentTimeMillis() - start) / 1000));
}
} catch (java.io.EOFException e) {
log.info("Likely the Unexpected end of ZLIB input. Likely similar to https://stackoverflow.com/q/55608979. Ignoring for now. Actual exception message: " + e.getMessage());
} finally {
br.close();
}
shardVsDocs.forEach((shard, docs) -> executor.submit(new UploadDocs(docs,
httpClient,
shardVsLeader.get(shard), tasks, completed)));
} finally {
executor.shutdown();
executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
httpClient.close();
}
HttpSolrClient client = new HttpSolrClient.Builder(baseUrl).build();
client.commit(collection);
client.close();

log.info("Indexed " + count + " docs." + "time taken : " + ((System.currentTimeMillis() - start) / 1000));
}


static class UploadDocs implements Runnable {

final List<String> docs;
final HttpClient client;
final String leaderUrl;
final AtomicInteger counter;
final AtomicInteger completed; // number of batches completed

UploadDocs(List<String> docs, HttpClient client, String leaderUrl, AtomicInteger counter, AtomicInteger completed) {
this.docs = docs;
this.client = client;
this.leaderUrl = leaderUrl;
this.counter = counter;
counter.incrementAndGet();
this.completed = completed;
}

@Override
public void run() {
HttpPost httpPost = new HttpPost(leaderUrl);
httpPost.setHeader(new BasicHeader("Content-Type", "application/json; charset=UTF-8"));
httpPost.getParams().setParameter("overwrite", "false");

httpPost.setEntity(new BasicHttpEntity() {
@Override
public boolean isStreaming() {
return true;
}

@Override
public void writeTo(OutputStream outstream) throws IOException {
OutputStreamWriter writer = new OutputStreamWriter(outstream);
for (String doc : docs) {
writer.append(doc).append('\n');
}
writer.flush();
}
});

try {
HttpResponse rsp = client.execute(httpPost);
int statusCode = rsp.getStatusLine().getStatusCode();
if (statusCode != 200) {
log.error("Failed a request: " +
rsp.getStatusLine() + " " + EntityUtils.toString(rsp.getEntity(), StandardCharsets.UTF_8));
}

} catch (IOException e) {
log.error("Error in request to url : " + leaderUrl, e);
} finally {
counter.decrementAndGet();
completed.incrementAndGet();
}


if (completed.get() % 100 == 0) log.info("\tBatches indexed: "+completed.get()+", currently queued: "+counter.get());
}
}
}
Loading