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

Cleanup and remove some old config options #723

Merged
merged 1 commit into from
Sep 19, 2024
Merged
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -7,9 +7,6 @@ indexDir: "/user/app/primary_index_base"
threadPoolConfiguration:
maxSearchingThreads: 4
maxIndexingThreads: 18
fileSendDelay: false
botoCfgPath: "/user/app/boto.cfg"
bucketName: "nrtsearch-bucket"
serviceName: "nrtsearch-service-test"
restoreState: False
downloadAsStream: "true"
serviceName: "nrtsearch-service-test"
Original file line number Diff line number Diff line change
Expand Up @@ -10,5 +10,4 @@ threadPoolConfiguration:
botoCfgPath: "/user/app/boto.cfg"
bucketName: "nrtsearch-bucket"
serviceName: "nrtsearch-service-test"
restoreState: False
downloadAsStream: "true"
24 changes: 3 additions & 21 deletions docs/server_configuration.rst
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,9 @@ Example server configuration
threadPoolConfiguration:
maxSearchingThreads: 4
maxIndexingThreads: 18
fileSendDelay: false
botoCfgPath: "/user/app/boto.cfg"
bucketName: "nrtsearch-bucket"
serviceName: "nrtsearch-service-test"
restoreState: False
downloadAsStream: "true"


.. list-table:: `LuceneServerConfiguration <https://github.com/Yelp/nrtsearch/blob/master/src/main/java/com/yelp/nrtsearch/server/config/LuceneServerConfiguration.java>`_
Expand Down Expand Up @@ -83,20 +80,10 @@ Example server configuration
- Path to AWS credentials (if using S3 for remote storage); Will use the DefaultAWSCredentialsProviderChain if omitted.
- null

* - downloadAsStream
- bool
- If enabled, the content downloader will perform a streaming extraction of tar archives from remote storage to disk. Otherwise, the downloader will only extract after finishing downloading the archive to disk.
- true

* - restoreState
- bool
- Enables loading state from external storage on startup
- false

* - deadlineCancellation
- bool
- Enables gRPC deadline based cancellation of requests. A request is cancelled early if it exceeds the deadline. Currently only supported by the search endpoint.
- false
- true

* - lowPriorityCopyPercentage
- int
Expand All @@ -109,15 +96,10 @@ Example server configuration
- []

* - pluginSearchPath
- str
- Search paths for plugins. These paths are separated by the system path separator character (; on Windows, : on Mac and Unix). The server will try to find the first directory in the search path matching a given plugin.
- str or list
- Search paths for plugins. This can either be a single string or a list of strings. The server will try to find the first directory in the search path containing a given plugin.
- plugins

* - publishJvmMetrics
- bool
- If enabled, registers JVM metrics with prometheus.
- true

* - useKeepAliveForReplication
- bool
- If enabled, the primary will enable keepAlive on the replication channel with keepAliveTime 1 minute and keepAliveTimeout 10 seconds. Replicas ignore this option.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.io.InputStream;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
Expand Down Expand Up @@ -75,17 +76,13 @@ public class LuceneServerConfiguration {
private final String bucketName;
private final int maxS3ClientRetries;
private final double[] metricsBuckets;
private final boolean publishJvmMetrics;
private final String[] plugins;
private final String pluginSearchPath;
private final List<String> pluginSearchPath;
private final String serviceName;
private final boolean restoreState;
private final ThreadPoolConfiguration threadPoolConfiguration;
private final IndexPreloadConfig preloadConfig;
private final QueryCacheConfig queryCacheConfig;
private final WarmerConfig warmerConfig;
private final boolean downloadAsStream;
private final boolean fileSendDelay;
private final boolean virtualSharding;
private final boolean decInitialCommit;
private final boolean syncInitialNrtPoint;
Expand Down Expand Up @@ -147,17 +144,16 @@ public LuceneServerConfiguration(InputStream yamlStream) {
metricsBuckets = DEFAULT_METRICS_BUCKETS;
}
this.metricsBuckets = metricsBuckets;
publishJvmMetrics = configReader.getBoolean("publishJvmMetrics", true);
plugins = configReader.getStringList("plugins", DEFAULT_PLUGINS).toArray(new String[0]);
pluginSearchPath =
configReader.getString("pluginSearchPath", DEFAULT_PLUGIN_SEARCH_PATH.toString());
configReader.get(
"pluginSearchPath",
LuceneServerConfiguration::getPluginSearchPath,
List.of(DEFAULT_PLUGIN_SEARCH_PATH.toString()));
serviceName = configReader.getString("serviceName", DEFAULT_SERVICE_NAME);
restoreState = configReader.getBoolean("restoreState", false);
preloadConfig = IndexPreloadConfig.fromConfig(configReader);
queryCacheConfig = QueryCacheConfig.fromConfig(configReader);
warmerConfig = WarmerConfig.fromConfig(configReader);
downloadAsStream = configReader.getBoolean("downloadAsStream", true);
fileSendDelay = configReader.getBoolean("fileSendDelay", false);
virtualSharding = configReader.getBoolean("virtualSharding", false);
decInitialCommit = configReader.getBoolean("decInitialCommit", true);
syncInitialNrtPoint = configReader.getBoolean("syncInitialNrtPoint", true);
Expand All @@ -169,7 +165,7 @@ public LuceneServerConfiguration(InputStream yamlStream) {
fileCopyConfig = FileCopyConfig.fromConfig(configReader);
threadPoolConfiguration = new ThreadPoolConfiguration(configReader);
scriptCacheConfig = ScriptCacheConfig.fromConfig(configReader);
deadlineCancellation = configReader.getBoolean("deadlineCancellation", false);
deadlineCancellation = configReader.getBoolean("deadlineCancellation", true);
stateConfig = StateConfig.fromConfig(configReader);
indexStartConfig = IndexStartConfig.fromConfig(configReader);
discoveryFileUpdateIntervalMs =
Expand Down Expand Up @@ -260,10 +256,6 @@ public double[] getMetricsBuckets() {
return metricsBuckets;
}

public boolean getPublishJvmMetrics() {
return publishJvmMetrics;
}

public int getReplicaReplicationPortPingInterval() {
return replicaReplicationPortPingInterval;
}
Expand All @@ -272,14 +264,10 @@ public String[] getPlugins() {
return this.plugins;
}

public String getPluginSearchPath() {
public List<String> getPluginSearchPath() {
return this.pluginSearchPath;
}

public boolean getRestoreState() {
return restoreState;
}

public IndexPreloadConfig getPreloadConfig() {
return preloadConfig;
}
Expand All @@ -292,14 +280,6 @@ public WarmerConfig getWarmerConfig() {
return warmerConfig;
}

public boolean getDownloadAsStream() {
return downloadAsStream;
}

public boolean getFileSendDelay() {
return fileSendDelay;
}

public boolean getVirtualSharding() {
return virtualSharding;
}
Expand Down Expand Up @@ -424,4 +404,16 @@ public long getMaxConnectionAgeForReplication() {
public long getMaxConnectionAgeGraceForReplication() {
return maxConnectionAgeGraceForReplication;
}

private static List<String> getPluginSearchPath(Object o) {
List<String> paths = new ArrayList<>();
if (o instanceof List list) {
for (Object item : list) {
paths.add(item.toString());
}
} else {
paths.add(o.toString());
}
return paths;
}
}
28 changes: 1 addition & 27 deletions src/main/java/com/yelp/nrtsearch/server/grpc/LuceneServer.java
Original file line number Diff line number Diff line change
Expand Up @@ -217,9 +217,7 @@ private void blockUntilShutdown() throws InterruptedException {
/** Register prometheus metrics exposed by /status/metrics */
private void registerMetrics(GlobalState globalState) {
// register jvm metrics
if (luceneServerConfiguration.getPublishJvmMetrics()) {
DefaultExports.register(collectorRegistry);
}
DefaultExports.register(collectorRegistry);
// register thread pool metrics
new ThreadPoolCollector().register(collectorRegistry);
collectorRegistry.register(RejectionCounterWrapper.rejectionCounter);
Expand Down Expand Up @@ -1856,9 +1854,6 @@ public void recvRawFile(
.build();
rawFileChunkStreamObserver.onNext(rawFileChunk);
totalRead += chunkSize;
if (globalState.getConfiguration().getFileSendDelay()) {
randomDelay(ThreadLocalRandom.current());
}
}
// EOF
rawFileChunkStreamObserver.onCompleted();
Expand Down Expand Up @@ -1973,27 +1968,6 @@ private void maybeCloseFile() {
};
}

/**
* induces random delay between 1ms to 10ms (both inclusive). Without this excessive buffering
* happens in server/primary if its to fast compared to receiver/replica. This only happens when
* we backfill an entire index i.e. very high indexing throughput.
* https://github.com/grpc/grpc-java/issues/6426. Note that flow control only works with client
* streaming, whereas we are using unary calls. For unary calls, you can
*
* <p>use NettyServerBuilder.maxConcurrentCallsPerConnection to limit concurrent calls
*
* <p>slow down to respond so that each request takes a little longer to get response.
*
* <p>For client streaming, you can in addition do manual flow control.
*
* @param random
* @throws InterruptedException
*/
private void randomDelay(Random random) throws InterruptedException {
int val = random.nextInt(10);
Thread.sleep(val + 1);
}

@Override
public void recvCopyState(
CopyStateRequest request, StreamObserver<CopyState> responseObserver) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -103,9 +102,7 @@ public List<PluginDescriptor> getLoadedPluginDescriptors() {
* OS path separator.
*/
List<File> getPluginSearchPath() {
return Stream.of(config.getPluginSearchPath().split(File.pathSeparator))
.map(File::new)
.collect(Collectors.toList());
return config.getPluginSearchPath().stream().map(File::new).collect(Collectors.toList());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1268,8 +1268,8 @@ public static List<VirtualField> getQueryVirtualFields() {
}

@Test
public void testCancellationDefaultDisabled() {
assertFalse(DeadlineUtils.getCancellationEnabled());
public void testCancellationDefaultEnabled() {
assertTrue(DeadlineUtils.getCancellationEnabled());
}

public static void checkHits(SearchResponse.Hit hit) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
package com.yelp.nrtsearch.server.plugins;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;

import com.yelp.nrtsearch.server.config.LuceneServerConfiguration;
Expand Down Expand Up @@ -45,6 +46,14 @@ private LuceneServerConfiguration getConfigWithSearchPath(String path) {
return new LuceneServerConfiguration(new ByteArrayInputStream(config.getBytes()));
}

private LuceneServerConfiguration getConfigWithSearchPaths(String... paths) {
StringBuilder config = new StringBuilder("pluginSearchPath:").append("\n");
for (String path : paths) {
config.append(" - ").append(path).append("\n");
}
return new LuceneServerConfiguration(new ByteArrayInputStream(config.toString().getBytes()));
}

private CollectorRegistry getCollectorRegistry() {
return new CollectorRegistry();
}
Expand All @@ -60,14 +69,9 @@ public void testGetSinglePluginSearchPath() {

@Test
public void testGetMultiPluginSearchPath() {
String searchPath =
"some1/plugin1/path1"
+ File.pathSeparator
+ "some2/plugin2/path2"
+ File.pathSeparator
+ "some3/plugin3/path3"
+ File.pathSeparator;
LuceneServerConfiguration config = getConfigWithSearchPath(searchPath);
LuceneServerConfiguration config =
getConfigWithSearchPaths(
"some1/plugin1/path1", "some2/plugin2/path2", "some3/plugin3/path3");
PluginsService pluginsService = new PluginsService(config, null, getCollectorRegistry());
List<File> expectedPaths = new ArrayList<>();
expectedPaths.add(new File("some1/plugin1/path1"));
Expand Down Expand Up @@ -200,7 +204,7 @@ public void testGetPluginInstance() {
CollectorRegistry collectorRegistry = getCollectorRegistry();
PluginsService pluginsService = new PluginsService(getEmptyConfig(), null, collectorRegistry);
Plugin loadedPlugin = pluginsService.getPluginInstance(LoadTestPlugin.class);
assertEquals(null, ((LoadTestPlugin) loadedPlugin).collectorRegistry);
assertNull(((LoadTestPlugin) loadedPlugin).collectorRegistry);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,5 +7,4 @@ indexDir: "primary_index_base"
threadPoolConfiguration:
maxSearchingThreads: 4
maxIndexingThreads: 18
fileSendDelay: false
verifyReplicationIndexId: false
Loading