summaryrefslogtreecommitdiffstats
path: root/container-search
diff options
context:
space:
mode:
authorLester Solbakken <lester.solbakken@gmail.com>2024-04-10 17:05:31 +0200
committerLester Solbakken <lester.solbakken@gmail.com>2024-04-10 17:05:31 +0200
commit148aa6016934d7075b4bb125ee40aec7d83394f7 (patch)
tree8126373eb8a3b66f8ea66288a3dc2f892c901912 /container-search
parent4d0144a4d249df6cce37539cba13969e9fd4ca4f (diff)
Add local LLM client and wire in container-llama
Diffstat (limited to 'container-search')
-rw-r--r--container-search/abi-spec.json91
-rw-r--r--container-search/pom.xml6
-rw-r--r--container-search/src/main/java/ai/vespa/llm/clients/LocalLLM.java132
-rwxr-xr-xcontainer-search/src/main/java/ai/vespa/search/llm/LLMSearcher.java77
-rwxr-xr-xcontainer-search/src/main/resources/configdefinitions/llm-local-client.def32
-rw-r--r--container-search/src/test/java/ai/vespa/llm/clients/LocalLLMTest.java236
6 files changed, 564 insertions, 10 deletions
diff --git a/container-search/abi-spec.json b/container-search/abi-spec.json
index e74fe22c588..8170d6bd9a8 100644
--- a/container-search/abi-spec.json
+++ b/container-search/abi-spec.json
@@ -9229,6 +9229,97 @@
"public static final java.lang.String[] CONFIG_DEF_SCHEMA"
]
},
+ "ai.vespa.llm.clients.LlmLocalClientConfig$Builder" : {
+ "superClass" : "java.lang.Object",
+ "interfaces" : [
+ "com.yahoo.config.ConfigInstance$Builder"
+ ],
+ "attributes" : [
+ "public",
+ "final"
+ ],
+ "methods" : [
+ "public void <init>()",
+ "public void <init>(ai.vespa.llm.clients.LlmLocalClientConfig)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder modelUrl(com.yahoo.config.UrlReference)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder localLlmFile(java.lang.String)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder parallelRequests(int)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder maxQueueSize(int)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder useGpu(boolean)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder gpuLayers(int)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder threads(int)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder contextSize(int)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig$Builder maxTokens(int)",
+ "public final boolean dispatchGetConfig(com.yahoo.config.ConfigInstance$Producer)",
+ "public final java.lang.String getDefMd5()",
+ "public final java.lang.String getDefName()",
+ "public final java.lang.String getDefNamespace()",
+ "public final boolean getApplyOnRestart()",
+ "public final void setApplyOnRestart(boolean)",
+ "public ai.vespa.llm.clients.LlmLocalClientConfig build()"
+ ],
+ "fields" : [ ]
+ },
+ "ai.vespa.llm.clients.LlmLocalClientConfig$Producer" : {
+ "superClass" : "java.lang.Object",
+ "interfaces" : [
+ "com.yahoo.config.ConfigInstance$Producer"
+ ],
+ "attributes" : [
+ "public",
+ "interface",
+ "abstract"
+ ],
+ "methods" : [
+ "public abstract void getConfig(ai.vespa.llm.clients.LlmLocalClientConfig$Builder)"
+ ],
+ "fields" : [ ]
+ },
+ "ai.vespa.llm.clients.LlmLocalClientConfig" : {
+ "superClass" : "com.yahoo.config.ConfigInstance",
+ "interfaces" : [ ],
+ "attributes" : [
+ "public",
+ "final"
+ ],
+ "methods" : [
+ "public static java.lang.String getDefMd5()",
+ "public static java.lang.String getDefName()",
+ "public static java.lang.String getDefNamespace()",
+ "public void <init>(ai.vespa.llm.clients.LlmLocalClientConfig$Builder)",
+ "public java.io.File modelUrl()",
+ "public java.lang.String localLlmFile()",
+ "public int parallelRequests()",
+ "public int maxQueueSize()",
+ "public boolean useGpu()",
+ "public int gpuLayers()",
+ "public int threads()",
+ "public int contextSize()",
+ "public int maxTokens()"
+ ],
+ "fields" : [
+ "public static final java.lang.String CONFIG_DEF_MD5",
+ "public static final java.lang.String CONFIG_DEF_NAME",
+ "public static final java.lang.String CONFIG_DEF_NAMESPACE",
+ "public static final java.lang.String[] CONFIG_DEF_SCHEMA"
+ ]
+ },
+ "ai.vespa.llm.clients.LocalLLM" : {
+ "superClass" : "com.yahoo.component.AbstractComponent",
+ "interfaces" : [
+ "ai.vespa.llm.LanguageModel"
+ ],
+ "attributes" : [
+ "public"
+ ],
+ "methods" : [
+ "public void <init>(ai.vespa.llm.clients.LlmLocalClientConfig)",
+ "public void deconstruct()",
+ "public java.util.List complete(ai.vespa.llm.completion.Prompt, ai.vespa.llm.InferenceParameters)",
+ "public java.util.concurrent.CompletableFuture completeAsync(ai.vespa.llm.completion.Prompt, ai.vespa.llm.InferenceParameters, java.util.function.Consumer)"
+ ],
+ "fields" : [ ]
+ },
"ai.vespa.llm.clients.OpenAI" : {
"superClass" : "ai.vespa.llm.clients.ConfigurableLanguageModel",
"interfaces" : [ ],
diff --git a/container-search/pom.xml b/container-search/pom.xml
index 5e7c60d49c3..38a4cb0ac2d 100644
--- a/container-search/pom.xml
+++ b/container-search/pom.xml
@@ -87,6 +87,12 @@
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>com.yahoo.vespa</groupId>
+ <artifactId>container-llama</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
<dependency>
<groupId>xerces</groupId>
diff --git a/container-search/src/main/java/ai/vespa/llm/clients/LocalLLM.java b/container-search/src/main/java/ai/vespa/llm/clients/LocalLLM.java
new file mode 100644
index 00000000000..e34f914729c
--- /dev/null
+++ b/container-search/src/main/java/ai/vespa/llm/clients/LocalLLM.java
@@ -0,0 +1,132 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package ai.vespa.llm.clients;
+
+import ai.vespa.llm.InferenceParameters;
+import ai.vespa.llm.LanguageModel;
+import ai.vespa.llm.completion.Completion;
+import ai.vespa.llm.completion.Prompt;
+import com.yahoo.component.AbstractComponent;
+import com.yahoo.component.annotation.Inject;
+import de.kherud.llama.LlamaModel;
+import de.kherud.llama.ModelParameters;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.logging.Logger;
+
+/**
+ * A language model running locally on the container node.
+ *
+ * @author lesters
+ */
+public class LocalLLM extends AbstractComponent implements LanguageModel {
+
+ private final static Logger logger = Logger.getLogger(LocalLLM.class.getName());
+ private final LlamaModel model;
+ private final ThreadPoolExecutor executor;
+ private final int contextSize;
+ private final int maxTokens;
+
+ @Inject
+ public LocalLLM(LlmLocalClientConfig config) {
+ this.executor = new ThreadPoolExecutor(config.parallelRequests(), config.parallelRequests(),
+ 0L, TimeUnit.MILLISECONDS,
+ config.maxQueueSize() > 0 ? new ArrayBlockingQueue<>(config.maxQueueSize()) : new SynchronousQueue<>(),
+ new ThreadPoolExecutor.AbortPolicy());
+
+ String modelFile;
+ if ( ! config.localLlmFile().isEmpty()) { // for testing
+ modelFile = config.localLlmFile();
+ } else if (config.modelUrl().exists()){
+ modelFile = config.modelUrl().getAbsolutePath();
+ } else {
+ throw new IllegalArgumentException("Local LLM model not set. " +
+ "Either set 'localLlmFile' or 'modelUrl' in 'llm-local-client' config.");
+ }
+
+ // Maximum number of tokens to generate - need this since some models can just generate infinitely
+ maxTokens = config.maxTokens();
+
+ // Only used if GPU is not used
+ var defaultThreadCount = Runtime.getRuntime().availableProcessors() - 2;
+
+ var modelParams = new ModelParameters()
+ .setModelFilePath(modelFile)
+ .setContinuousBatching(true)
+ .setNParallel(config.parallelRequests())
+ .setNThreads(config.threads() <= 0 ? defaultThreadCount : config.threads())
+ .setNCtx(config.contextSize())
+ .setNGpuLayers(config.useGpu() ? config.gpuLayers() : 0);
+
+ long startLoad = System.nanoTime();
+ model = new LlamaModel(modelParams);
+ long loadTime = System.nanoTime() - startLoad;
+ logger.info(String.format("Loaded model %s in %.2f sec", modelFile, (loadTime*1.0/1000000000)));
+
+ // Todo: handle prompt context size - such as give a warning when prompt exceeds context size
+ this.contextSize = config.contextSize();
+ }
+
+ @Override
+ public void deconstruct() {
+ logger.info("Closing LLM model...");
+ model.close();
+ executor.shutdownNow();
+ }
+
+ @Override
+ public List<Completion> complete(Prompt prompt, InferenceParameters options) {
+ StringBuilder result = new StringBuilder();
+ var future = completeAsync(prompt, options, completion -> {
+ result.append(completion.text());
+ }).exceptionally(exception -> Completion.FinishReason.error);
+ var reason = future.join();
+
+ List<Completion> completions = new ArrayList<>();
+ completions.add(new Completion(result.toString(), reason));
+ return completions;
+ }
+
+ @Override
+ public CompletableFuture<Completion.FinishReason> completeAsync(Prompt prompt, InferenceParameters options, Consumer<Completion> consumer) {
+ var inferParams = new de.kherud.llama.InferenceParameters(prompt.asString().stripLeading());
+
+ // We always set this to some value to avoid infinite token generation
+ inferParams.setNPredict(maxTokens);
+
+ options.ifPresent("temperature", (v) -> inferParams.setTemperature(Float.parseFloat(v)));
+ options.ifPresent("topk", (v) -> inferParams.setTopK(Integer.parseInt(v)));
+ options.ifPresent("topp", (v) -> inferParams.setTopP(Integer.parseInt(v)));
+ options.ifPresent("npredict", (v) -> inferParams.setNPredict(Integer.parseInt(v)));
+ options.ifPresent("repeatpenalty", (v) -> inferParams.setRepeatPenalty(Float.parseFloat(v)));
+ // Todo: more options?
+
+ var completionFuture = new CompletableFuture<Completion.FinishReason>();
+ try {
+ executor.submit(() -> {
+ for (LlamaModel.Output output : model.generate(inferParams)) {
+ consumer.accept(Completion.from(output.text, Completion.FinishReason.none));
+ }
+ completionFuture.complete(Completion.FinishReason.stop);
+ });
+ } catch (RejectedExecutionException e) {
+ // If we have too many requests (active + any waiting in queue), we reject the completion
+ int activeCount = executor.getActiveCount();
+ int queueSize = executor.getQueue().size();
+ String error = String.format("Rejected completion due to too many requests, " +
+ "%d active, %d in queue", activeCount, queueSize);
+ logger.info(error);
+ consumer.accept(Completion.from(error, Completion.FinishReason.error));
+ completionFuture.complete(Completion.FinishReason.error);
+ }
+ return completionFuture;
+ }
+
+}
diff --git a/container-search/src/main/java/ai/vespa/search/llm/LLMSearcher.java b/container-search/src/main/java/ai/vespa/search/llm/LLMSearcher.java
index 860fc69af91..951b3ae3de9 100755
--- a/container-search/src/main/java/ai/vespa/search/llm/LLMSearcher.java
+++ b/container-search/src/main/java/ai/vespa/search/llm/LLMSearcher.java
@@ -86,24 +86,34 @@ public class LLMSearcher extends Searcher {
return stream ? completeAsync(query, prompt, options) : completeSync(query, prompt, options);
}
+ private boolean shouldAddPrompt(Query query) {
+ return query.getTrace().getLevel() >= 1;
+ }
+
+ private boolean shouldAddTokenStats(Query query) {
+ return query.getTrace().getLevel() >= 1;
+ }
+
private Result completeAsync(Query query, Prompt prompt, InferenceParameters options) {
- EventStream eventStream = new EventStream();
+ final EventStream eventStream = new EventStream();
- if (query.getTrace().getLevel() >= 1) {
+ if (shouldAddPrompt(query)) {
eventStream.add(prompt.asString(), "prompt");
}
- languageModel.completeAsync(prompt, options, token -> {
- eventStream.add(token.text());
+ final TokenStats tokenStats = new TokenStats();
+ languageModel.completeAsync(prompt, options, completion -> {
+ tokenStats.onToken();
+ handleCompletion(eventStream, completion);
}).exceptionally(exception -> {
- int errorCode = 400;
- if (exception instanceof LanguageModelException languageModelException) {
- errorCode = languageModelException.code();
- }
- eventStream.error(languageModelId, new ErrorMessage(errorCode, exception.getMessage()));
+ handleException(eventStream, exception);
eventStream.markComplete();
return Completion.FinishReason.error;
}).thenAccept(finishReason -> {
+ tokenStats.onCompletion();
+ if (shouldAddTokenStats(query)) {
+ eventStream.add(tokenStats.report(), "stats");
+ }
eventStream.markComplete();
});
@@ -112,10 +122,26 @@ public class LLMSearcher extends Searcher {
return new Result(query, hitGroup);
}
+ private void handleCompletion(EventStream eventStream, Completion completion) {
+ if (completion.finishReason() == Completion.FinishReason.error) {
+ eventStream.add(completion.text(), "error");
+ } else {
+ eventStream.add(completion.text());
+ }
+ }
+
+ private void handleException(EventStream eventStream, Throwable exception) {
+ int errorCode = 400;
+ if (exception instanceof LanguageModelException languageModelException) {
+ errorCode = languageModelException.code();
+ }
+ eventStream.error(languageModelId, new ErrorMessage(errorCode, exception.getMessage()));
+ }
+
private Result completeSync(Query query, Prompt prompt, InferenceParameters options) {
EventStream eventStream = new EventStream();
- if (query.getTrace().getLevel() >= 1) {
+ if (shouldAddPrompt(query)) {
eventStream.add(prompt.asString(), "prompt");
}
@@ -169,4 +195,35 @@ public class LLMSearcher extends Searcher {
return lookupPropertyWithOrWithoutPrefix(API_KEY_HEADER, p -> query.getHttpRequest().getHeader(p));
}
+ private static class TokenStats {
+
+ private long start;
+ private long timeToFirstToken;
+ private long timeToLastToken;
+ private long tokens = 0;
+
+ TokenStats() {
+ start = System.currentTimeMillis();
+ }
+
+ void onToken() {
+ if (tokens == 0) {
+ timeToFirstToken = System.currentTimeMillis() - start;
+ }
+ tokens++;
+ }
+
+ void onCompletion() {
+ timeToLastToken = System.currentTimeMillis() - start;
+ }
+
+ String report() {
+ return "Time to first token: " + timeToFirstToken + " ms, " +
+ "Generation time: " + timeToLastToken + " ms, " +
+ "Generated tokens: " + tokens + " " +
+ String.format("(%.2f tokens/sec)", tokens / (timeToLastToken / 1000.0));
+ }
+
+ }
+
}
diff --git a/container-search/src/main/resources/configdefinitions/llm-local-client.def b/container-search/src/main/resources/configdefinitions/llm-local-client.def
new file mode 100755
index 00000000000..08eab19f0f8
--- /dev/null
+++ b/container-search/src/main/resources/configdefinitions/llm-local-client.def
@@ -0,0 +1,32 @@
+# Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package=ai.vespa.llm.clients
+
+# Url to the model to use
+modelUrl url default=""
+
+# Local file path to the model to use - will have precedence over model_url if set - mostly for testing
+localLlmFile string default=""
+
+# Maximum number of requests to handle in parallel pr container node
+parallelRequests int default=10
+
+# Additional number of requests to put in queue for processing before starting to reject new requests
+maxQueueSize int default=10
+
+# Use GPU
+useGpu bool default=false
+
+# Maximum number of model layers to run on GPU
+gpuLayers int default=1000000
+
+# Number of threads to use for CPU processing - -1 means use all available cores
+# Not used for GPU processing
+threads int default=-1
+
+# Context size for the model
+# Context is divided between parallel requests. So for 10 parallel requests, each "slot" gets 1/10 of the context
+contextSize int default=512
+
+# Maximum number of tokens to process in one request - overriden by inference parameters
+maxTokens int default=512
+
diff --git a/container-search/src/test/java/ai/vespa/llm/clients/LocalLLMTest.java b/container-search/src/test/java/ai/vespa/llm/clients/LocalLLMTest.java
new file mode 100644
index 00000000000..7dc32b6c929
--- /dev/null
+++ b/container-search/src/test/java/ai/vespa/llm/clients/LocalLLMTest.java
@@ -0,0 +1,236 @@
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+package ai.vespa.llm.clients;
+
+import ai.vespa.llm.InferenceParameters;
+import ai.vespa.llm.completion.Completion;
+import ai.vespa.llm.completion.Prompt;
+import ai.vespa.llm.completion.StringPrompt;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for LocalLLM. Tests are disabled due to size of models.
+ *
+ * @author lesters
+ */
+public class LocalLLMTest {
+
+ // Download a model - for instance https://huggingface.co/TheBloke/Mistral-7B-Instruct-v0.2-GGUF/resolve/main/mistral-7b-instruct-v0.2.Q6_K.gguf
+ private static String model = "path/to/model";
+
+ @Test
+ @Disabled
+ public void testGeneration() {
+ var prompt = StringPrompt.from("Why are ducks better than cats? Be concise, " +
+ "but use the word 'spoon' somewhere in your answer.");
+ var llm = createLLM(model);
+ var result = llm.complete(prompt, defaultOptions());
+ assertEquals(Completion.FinishReason.stop, result.get(0).finishReason());
+ assertTrue(result.get(0).text().contains("spoon"));
+ llm.deconstruct();
+ }
+
+ @Test
+ @Disabled
+ public void testAsyncGeneration() {
+ var sb = new StringBuilder();
+ Prompt prompt = StringPrompt.from(testContext() + "What was the Manhattan Project? Only use the documents given here as reference.");
+
+ var config = new LlmLocalClientConfig.Builder()
+ .useGpu(true)
+ .parallelRequests(1)
+ .contextSize(1024)
+ .localLlmFile(model);
+ var llm = new LocalLLM(config.build());
+
+ var future = llm.completeAsync(prompt, defaultOptions(), completion -> {
+ sb.append(completion.text());
+ System.out.print(completion.text());
+ }).exceptionally(exception -> Completion.FinishReason.error);
+
+ assertFalse(future.isDone());
+ var reason = future.join();
+ assertTrue(future.isDone());
+ assertNotEquals(reason, Completion.FinishReason.error);
+
+ System.out.println(prompt.asString());
+ System.out.println(sb);
+
+ llm.deconstruct();
+ }
+
+ @Test
+ @Disabled
+ public void testParallelGeneration() {
+ var prompts = testPrompts();
+ var promptsToUse = prompts.size();
+ var parallelRequests = 10;
+
+ var futures = new ArrayList<CompletableFuture<Completion.FinishReason>>(Collections.nCopies(promptsToUse, null));
+ var completions = new ArrayList<StringBuilder>(Collections.nCopies(promptsToUse, null));
+
+ var config = new LlmLocalClientConfig.Builder()
+ .useGpu(true)
+ .parallelRequests(parallelRequests)
+ .localLlmFile(model);
+ var llm = new LocalLLM(config.build());
+
+ var start = System.currentTimeMillis();
+ for (int i = 0; i < promptsToUse; i++) {
+ final var seq = i;
+
+ completions.set(seq, new StringBuilder());
+ futures.set(seq, llm.completeAsync(StringPrompt.from(prompts.get(seq)), defaultOptions(), completion -> {
+ completions.get(seq).append(completion.text());
+ }).exceptionally(exception -> Completion.FinishReason.error));
+ }
+ for (var future : futures) {
+ var reason = future.join();
+ assertNotEquals(reason, Completion.FinishReason.error);
+ }
+ for (int i = 0; i < promptsToUse; i++) {
+ var reason = futures.get(i).join();
+ assertNotEquals(reason, Completion.FinishReason.error);
+ System.out.println("\n\n***\n" + prompts.get(i) + ":\n***\n" + completions.get(i));
+ }
+ System.out.println("Time: " + (System.currentTimeMillis() - start) / 1000.0 + "s");
+
+ llm.deconstruct();
+ }
+
+ @Test
+ @Disabled
+ public void testRejection() {
+ var prompts = testPrompts();
+ var promptsToUse = prompts.size();
+ var parallelRequests = 1;
+ var additionalQueue = 0;
+ // 9 should be rejected
+
+ var futures = new ArrayList<CompletableFuture<Completion.FinishReason>>(Collections.nCopies(promptsToUse, null));
+ var completions = new ArrayList<StringBuilder>(Collections.nCopies(promptsToUse, null));
+
+ var config = new LlmLocalClientConfig.Builder()
+ .useGpu(true)
+ .parallelRequests(parallelRequests)
+ .maxQueueSize(additionalQueue)
+ .localLlmFile(model);
+ var llm = new LocalLLM(config.build());
+
+ final AtomicInteger rejected = new AtomicInteger(0);
+ for (int i = 0; i < promptsToUse; i++) {
+ final var seq = i;
+
+ completions.set(seq, new StringBuilder());
+ var future = llm.completeAsync(StringPrompt.from(prompts.get(seq)), defaultOptions(), completion -> {
+ completions.get(seq).append(completion.text());
+ if (completion.finishReason() == Completion.FinishReason.error) {
+ rejected.incrementAndGet();
+ }
+ }).exceptionally(exception -> Completion.FinishReason.error);
+ futures.set(seq, future);
+ }
+ for (int i = 0; i < promptsToUse; i++) {
+ futures.get(i).join();
+ System.out.println("\n\n***\n" + prompts.get(i) + ":\n***\n" + completions.get(i));
+ }
+
+ assertEquals(9, rejected.get());
+ llm.deconstruct();
+ }
+
+ private static InferenceParameters defaultOptions() {
+ final Map<String, String> options = Map.of(
+ "temperature", "0.1",
+ "npredict", "100"
+ );
+ return new InferenceParameters(options::get);
+ }
+
+ private List<String> testPrompts() {
+ List<String> prompts = new ArrayList<>();
+ prompts.add("Write a short story about a time-traveling detective who must solve a mystery that spans multiple centuries.");
+ prompts.add("Explain the concept of blockchain technology and its implications for data security in layman's terms.");
+ prompts.add("Discuss the socio-economic impacts of the Industrial Revolution in 19th century Europe.");
+ prompts.add("Describe a future where humans have colonized Mars, focusing on daily life and societal structure.");
+ prompts.add("Analyze the statement 'If a tree falls in a forest and no one is around to hear it, does it make a sound?' from both a philosophical and a physics perspective.");
+ prompts.add("Translate the following sentence into French: 'The quick brown fox jumps over the lazy dog.'");
+ prompts.add("Explain what the following Python code does: `print([x for x in range(10) if x % 2 == 0])`.");
+ prompts.add("Provide general guidelines for maintaining a healthy lifestyle to reduce the risk of developing heart disease.");
+ prompts.add("Create a detailed description of a fictional planet, including its ecosystem, dominant species, and technology level.");
+ prompts.add("Discuss the impact of social media on interpersonal communication in the 21st century.");
+ return prompts;
+ }
+
+ private static String testContext() {
+ return "sddocname: passage\n" +
+ "id: 2\n" +
+ "text: Essay on The Manhattan Project - The Manhattan Project The Manhattan Project was to see if making an atomic bomb possible. The success of this project would forever change the world forever making it known that something this powerful can be manmade.\n" +
+ "documentid: id:msmarco:passage::2\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 0\n" +
+ "text: The presence of communication amid scientific minds was equally important to the success of the Manhattan Project as scientific intellect was. The only cloud hanging over the impressive achievement of the atomic researchers and engineers is what their success truly meant; hundreds of thousands of innocent lives obliterated.\n" +
+ "documentid: id:msmarco:passage::0\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 7\n" +
+ "text: Manhattan Project. The Manhattan Project was a research and development undertaking during World War II that produced the first nuclear weapons. It was led by the United States with the support of the United Kingdom and Canada. From 1942 to 1946, the project was under the direction of Major General Leslie Groves of the U.S. Army Corps of Engineers. Nuclear physicist Robert Oppenheimer was the director of the Los Alamos Laboratory that designed the actual bombs. The Army component of the project was designated the\n" +
+ "documentid: id:msmarco:passage::7\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 3\n" +
+ "text: The Manhattan Project was the name for a project conducted during World War II, to develop the first atomic bomb. It refers specifically to the period of the project from 194 … 2-1946 under the control of the U.S. Army Corps of Engineers, under the administration of General Leslie R. Groves.\n" +
+ "documentid: id:msmarco:passage::3\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 9\n" +
+ "text: One of the main reasons Hanford was selected as a site for the Manhattan Project's B Reactor was its proximity to the Columbia River, the largest river flowing into the Pacific Ocean from the North American coast.\n" +
+ "documentid: id:msmarco:passage::9\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 5\n" +
+ "text: The Manhattan Project. This once classified photograph features the first atomic bomb — a weapon that atomic scientists had nicknamed Gadget.. The nuclear age began on July 16, 1945, when it was detonated in the New Mexico desert.\n" +
+ "documentid: id:msmarco:passage::5\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 8\n" +
+ "text: In June 1942, the United States Army Corps of Engineers began the Manhattan Project- The secret name for the 2 atomic bombs.\n" +
+ "documentid: id:msmarco:passage::8\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 1\n" +
+ "text: The Manhattan Project and its atomic bomb helped bring an end to World War II. Its legacy of peaceful uses of atomic energy continues to have an impact on history and science.\n" +
+ "documentid: id:msmarco:passage::1\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 6\n" +
+ "text: Nor will it attempt to substitute for the extraordinarily rich literature on the atomic bombs and the end of World War II. This collection does not attempt to document the origins and development of the Manhattan Project.\n" +
+ "documentid: id:msmarco:passage::6\n" +
+ "\n" +
+ "sddocname: passage\n" +
+ "id: 4\n" +
+ "text: versions of each volume as well as complementary websites. The first website–The Manhattan Project: An Interactive History–is available on the Office of History and Heritage Resources website, http://www.cfo. doe.gov/me70/history. The Office of History and Heritage Resources and the National Nuclear Security\n" +
+ "documentid: id:msmarco:passage::4\n" +
+ "\n" +
+ "\n";
+ }
+
+ private static LocalLLM createLLM(String modelPath) {
+ var config = new LlmLocalClientConfig.Builder().localLlmFile(modelPath).build();
+ return new LocalLLM(config);
+ }
+
+}