repositoriesServiceSupplier
) {
+ int urgentEventLoopThreads = urgentPoolCount(clusterService.getSettings());
int priorityEventLoopThreads = priorityPoolCount(clusterService.getSettings());
int normalEventLoopThreads = normalPoolCount(clusterService.getSettings());
+ this.urgentExecutorBuilder = new AsyncExecutorContainer(
+ threadPool.executor(URGENT_FUTURE_COMPLETION),
+ threadPool.executor(URGENT_STREAM_READER),
+ new AsyncTransferEventLoopGroup(urgentEventLoopThreads)
+ );
this.priorityExecutorBuilder = new AsyncExecutorContainer(
threadPool.executor(PRIORITY_FUTURE_COMPLETION),
threadPool.executor(PRIORITY_STREAM_READER),
@@ -170,7 +194,8 @@ protected S3Repository createRepository(
AsyncTransferManager asyncUploadUtils = new AsyncTransferManager(
S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.get(clusterService.getSettings()).getBytes(),
normalExecutorBuilder.getStreamReader(),
- priorityExecutorBuilder.getStreamReader()
+ priorityExecutorBuilder.getStreamReader(),
+ urgentExecutorBuilder.getStreamReader()
);
return new S3Repository(
metadata,
@@ -179,10 +204,12 @@ protected S3Repository createRepository(
clusterService,
recoverySettings,
asyncUploadUtils,
+ urgentExecutorBuilder,
priorityExecutorBuilder,
normalExecutorBuilder,
s3AsyncService,
- S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings())
+ S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings()),
+ configPath
);
}
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java
index 3a35f6135f28b..d7e47e0ab1bcc 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java
@@ -54,7 +54,7 @@
* Wrapper around an S3 object that will retry the {@link GetObjectRequest} if the download fails part-way through, resuming from where
* the failure occurred. This should be handled by the SDK but it isn't today. This should be revisited in the future (e.g. before removing
* the {@code LegacyESVersion#V_7_0_0} version constant) and removed when the SDK handles retries itself.
- *
+ *
* See https://github.com/aws/aws-sdk-java/issues/856 for the related SDK issue
*/
class S3RetryingInputStream extends InputStream {
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java
index b13672b4179f8..b1b3e19eac275 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java
@@ -90,6 +90,7 @@
import java.security.SecureRandom;
import java.time.Duration;
import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
import static java.util.Collections.emptyMap;
@@ -100,7 +101,7 @@ class S3Service implements Closeable {
private static final String DEFAULT_S3_ENDPOINT = "s3.amazonaws.com";
- private volatile Map clientsCache = emptyMap();
+ private volatile Map clientsCache = new ConcurrentHashMap<>();
/**
* Client settings calculated from static configuration and settings in the keystore.
@@ -111,7 +112,7 @@ class S3Service implements Closeable {
* Client settings derived from those in {@link #staticClientSettings} by combining them with settings
* in the {@link RepositoryMetadata}.
*/
- private volatile Map derivedClientSettings = emptyMap();
+ private volatile Map derivedClientSettings = new ConcurrentHashMap<>();
S3Service(final Path configPath) {
staticClientSettings = MapBuilder.newMapBuilder()
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/StatsMetricPublisher.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/StatsMetricPublisher.java
index cad0037f99249..0c63bfdb1ff97 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/StatsMetricPublisher.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/StatsMetricPublisher.java
@@ -8,10 +8,13 @@
package org.opensearch.repositories.s3;
-import software.amazon.awssdk.http.HttpMetric;
import software.amazon.awssdk.metrics.MetricCollection;
import software.amazon.awssdk.metrics.MetricPublisher;
+import software.amazon.awssdk.metrics.MetricRecord;
+import org.opensearch.common.blobstore.BlobStore;
+
+import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
@@ -20,18 +23,67 @@ public class StatsMetricPublisher {
private final Stats stats = new Stats();
+ private final Map extendedStats = new HashMap<>() {
+ {
+ put(BlobStore.Metric.REQUEST_LATENCY, new Stats());
+ put(BlobStore.Metric.REQUEST_SUCCESS, new Stats());
+ put(BlobStore.Metric.REQUEST_FAILURE, new Stats());
+ put(BlobStore.Metric.RETRY_COUNT, new Stats());
+ }
+ };
+
public MetricPublisher listObjectsMetricPublisher = new MetricPublisher() {
@Override
public void publish(MetricCollection metricCollection) {
- stats.listCount.addAndGet(
- metricCollection.children()
- .stream()
- .filter(
- metricRecords -> metricRecords.name().equals("ApiCallAttempt")
- && !metricRecords.metricValues(HttpMetric.HTTP_STATUS_CODE).isEmpty()
- )
- .count()
- );
+ for (MetricRecord> metricRecord : metricCollection) {
+ switch (metricRecord.metric().name()) {
+ case "ApiCallDuration":
+ extendedStats.get(BlobStore.Metric.REQUEST_LATENCY).listMetrics.addAndGet(
+ ((Duration) metricRecord.value()).toMillis()
+ );
+ break;
+ case "RetryCount":
+ extendedStats.get(BlobStore.Metric.RETRY_COUNT).listMetrics.addAndGet(((Integer) metricRecord.value()));
+ break;
+ case "ApiCallSuccessful":
+ if ((Boolean) metricRecord.value()) {
+ extendedStats.get(BlobStore.Metric.REQUEST_SUCCESS).listMetrics.addAndGet(1);
+ } else {
+ extendedStats.get(BlobStore.Metric.REQUEST_FAILURE).listMetrics.addAndGet(1);
+ }
+ stats.listMetrics.addAndGet(1);
+ break;
+ }
+ }
+ }
+
+ @Override
+ public void close() {}
+ };
+
+ public MetricPublisher deleteObjectsMetricPublisher = new MetricPublisher() {
+ @Override
+ public void publish(MetricCollection metricCollection) {
+ for (MetricRecord> metricRecord : metricCollection) {
+ switch (metricRecord.metric().name()) {
+ case "ApiCallDuration":
+ extendedStats.get(BlobStore.Metric.REQUEST_LATENCY).deleteMetrics.addAndGet(
+ ((Duration) metricRecord.value()).toMillis()
+ );
+ break;
+ case "RetryCount":
+ extendedStats.get(BlobStore.Metric.RETRY_COUNT).deleteMetrics.addAndGet(((Integer) metricRecord.value()));
+ break;
+ case "ApiCallSuccessful":
+ if ((Boolean) metricRecord.value()) {
+ extendedStats.get(BlobStore.Metric.REQUEST_SUCCESS).deleteMetrics.addAndGet(1);
+ } else {
+ extendedStats.get(BlobStore.Metric.REQUEST_FAILURE).deleteMetrics.addAndGet(1);
+ }
+ stats.deleteMetrics.addAndGet(1);
+ break;
+ }
+ }
}
@Override
@@ -41,15 +93,26 @@ public void close() {}
public MetricPublisher getObjectMetricPublisher = new MetricPublisher() {
@Override
public void publish(MetricCollection metricCollection) {
- stats.getCount.addAndGet(
- metricCollection.children()
- .stream()
- .filter(
- metricRecords -> metricRecords.name().equals("ApiCallAttempt")
- && !metricRecords.metricValues(HttpMetric.HTTP_STATUS_CODE).isEmpty()
- )
- .count()
- );
+ for (MetricRecord> metricRecord : metricCollection) {
+ switch (metricRecord.metric().name()) {
+ case "ApiCallDuration":
+ extendedStats.get(BlobStore.Metric.REQUEST_LATENCY).getMetrics.addAndGet(
+ ((Duration) metricRecord.value()).toMillis()
+ );
+ break;
+ case "RetryCount":
+ extendedStats.get(BlobStore.Metric.RETRY_COUNT).getMetrics.addAndGet(((Integer) metricRecord.value()));
+ break;
+ case "ApiCallSuccessful":
+ if ((Boolean) metricRecord.value()) {
+ extendedStats.get(BlobStore.Metric.REQUEST_SUCCESS).getMetrics.addAndGet(1);
+ } else {
+ extendedStats.get(BlobStore.Metric.REQUEST_FAILURE).getMetrics.addAndGet(1);
+ }
+ stats.getMetrics.addAndGet(1);
+ break;
+ }
+ }
}
@Override
@@ -59,15 +122,26 @@ public void close() {}
public MetricPublisher putObjectMetricPublisher = new MetricPublisher() {
@Override
public void publish(MetricCollection metricCollection) {
- stats.putCount.addAndGet(
- metricCollection.children()
- .stream()
- .filter(
- metricRecords -> metricRecords.name().equals("ApiCallAttempt")
- && !metricRecords.metricValues(HttpMetric.HTTP_STATUS_CODE).isEmpty()
- )
- .count()
- );
+ for (MetricRecord> metricRecord : metricCollection) {
+ switch (metricRecord.metric().name()) {
+ case "ApiCallDuration":
+ extendedStats.get(BlobStore.Metric.REQUEST_LATENCY).putMetrics.addAndGet(
+ ((Duration) metricRecord.value()).toMillis()
+ );
+ break;
+ case "RetryCount":
+ extendedStats.get(BlobStore.Metric.RETRY_COUNT).putMetrics.addAndGet(((Integer) metricRecord.value()));
+ break;
+ case "ApiCallSuccessful":
+ if ((Boolean) metricRecord.value()) {
+ extendedStats.get(BlobStore.Metric.REQUEST_SUCCESS).putMetrics.addAndGet(1);
+ } else {
+ extendedStats.get(BlobStore.Metric.REQUEST_FAILURE).putMetrics.addAndGet(1);
+ }
+ stats.putMetrics.addAndGet(1);
+ break;
+ }
+ }
}
@Override
@@ -77,15 +151,26 @@ public void close() {}
public MetricPublisher multipartUploadMetricCollector = new MetricPublisher() {
@Override
public void publish(MetricCollection metricCollection) {
- stats.postCount.addAndGet(
- metricCollection.children()
- .stream()
- .filter(
- metricRecords -> metricRecords.name().equals("ApiCallAttempt")
- && !metricRecords.metricValues(HttpMetric.HTTP_STATUS_CODE).isEmpty()
- )
- .count()
- );
+ for (MetricRecord> metricRecord : metricCollection) {
+ switch (metricRecord.metric().name()) {
+ case "ApiCallDuration":
+ extendedStats.get(BlobStore.Metric.REQUEST_LATENCY).multiPartPutMetrics.addAndGet(
+ ((Duration) metricRecord.value()).toMillis()
+ );
+ break;
+ case "RetryCount":
+ extendedStats.get(BlobStore.Metric.RETRY_COUNT).multiPartPutMetrics.addAndGet(((Integer) metricRecord.value()));
+ break;
+ case "ApiCallSuccessful":
+ if ((Boolean) metricRecord.value()) {
+ extendedStats.get(BlobStore.Metric.REQUEST_SUCCESS).multiPartPutMetrics.addAndGet(1);
+ } else {
+ extendedStats.get(BlobStore.Metric.REQUEST_FAILURE).multiPartPutMetrics.addAndGet(1);
+ }
+ stats.multiPartPutMetrics.addAndGet(1);
+ break;
+ }
+ }
}
@Override
@@ -96,22 +181,29 @@ public Stats getStats() {
return stats;
}
+ public Map getExtendedStats() {
+ return extendedStats;
+ }
+
static class Stats {
- final AtomicLong listCount = new AtomicLong();
+ final AtomicLong listMetrics = new AtomicLong();
+
+ final AtomicLong getMetrics = new AtomicLong();
- final AtomicLong getCount = new AtomicLong();
+ final AtomicLong putMetrics = new AtomicLong();
- final AtomicLong putCount = new AtomicLong();
+ final AtomicLong deleteMetrics = new AtomicLong();
- final AtomicLong postCount = new AtomicLong();
+ final AtomicLong multiPartPutMetrics = new AtomicLong();
Map toMap() {
final Map results = new HashMap<>();
- results.put("GetObject", getCount.get());
- results.put("ListObjects", listCount.get());
- results.put("PutObject", putCount.get());
- results.put("PutMultipartObject", postCount.get());
+ results.put("GetObject", getMetrics.get());
+ results.put("ListObjects", listMetrics.get());
+ results.put("PutObject", putMetrics.get());
+ results.put("DeleteObjects", deleteMetrics.get());
+ results.put("PutMultipartObject", multiPartPutMetrics.get());
return results;
}
}
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java
index ad6939ce299d6..933ee6dc29513 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java
@@ -23,10 +23,13 @@
import org.opensearch.common.StreamContext;
import org.opensearch.common.blobstore.stream.write.WritePriority;
import org.opensearch.common.io.InputStreamContainer;
+import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.repositories.s3.SocketAccess;
import org.opensearch.repositories.s3.io.CheckedContainer;
+import java.io.BufferedInputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
@@ -45,6 +48,7 @@ public class AsyncPartsHandler {
* @param s3AsyncClient S3 client to use for upload
* @param executorService Thread pool for regular upload
* @param priorityExecutorService Thread pool for priority uploads
+ * @param urgentExecutorService Thread pool for urgent uploads
* @param uploadRequest request for upload
* @param streamContext Stream context used in supplying individual file parts
* @param uploadId Upload Id against which multi-part is being performed
@@ -57,6 +61,7 @@ public static List> uploadParts(
S3AsyncClient s3AsyncClient,
ExecutorService executorService,
ExecutorService priorityExecutorService,
+ ExecutorService urgentExecutorService,
UploadRequest uploadRequest,
StreamContext streamContext,
String uploadId,
@@ -80,6 +85,7 @@ public static List> uploadParts(
s3AsyncClient,
executorService,
priorityExecutorService,
+ urgentExecutorService,
completedParts,
inputStreamContainers,
futures,
@@ -126,6 +132,7 @@ private static void uploadPart(
S3AsyncClient s3AsyncClient,
ExecutorService executorService,
ExecutorService priorityExecutorService,
+ ExecutorService urgentExecutorService,
AtomicReferenceArray completedParts,
AtomicReferenceArray inputStreamContainers,
List> futures,
@@ -135,29 +142,47 @@ private static void uploadPart(
) {
Integer partNumber = uploadPartRequest.partNumber();
- ExecutorService streamReadExecutor = uploadRequest.getWritePriority() == WritePriority.HIGH
- ? priorityExecutorService
- : executorService;
+ ExecutorService streamReadExecutor;
+ if (uploadRequest.getWritePriority() == WritePriority.URGENT) {
+ streamReadExecutor = urgentExecutorService;
+ } else if (uploadRequest.getWritePriority() == WritePriority.HIGH) {
+ streamReadExecutor = priorityExecutorService;
+ } else {
+ streamReadExecutor = executorService;
+ }
+ // Buffered stream is needed to allow mark and reset ops during IO errors so that only buffered
+ // data can be retried instead of retrying whole file by the application.
+ InputStream inputStream = new BufferedInputStream(inputStreamContainer.getInputStream(), (int) (ByteSizeUnit.MB.toBytes(1) + 1));
CompletableFuture uploadPartResponseFuture = SocketAccess.doPrivileged(
() -> s3AsyncClient.uploadPart(
uploadPartRequest,
- AsyncRequestBody.fromInputStream(
- inputStreamContainer.getInputStream(),
- inputStreamContainer.getContentLength(),
- streamReadExecutor
- )
+ AsyncRequestBody.fromInputStream(inputStream, inputStreamContainer.getContentLength(), streamReadExecutor)
)
);
- CompletableFuture convertFuture = uploadPartResponseFuture.thenApply(
- uploadPartResponse -> convertUploadPartResponse(
- completedParts,
- inputStreamContainers,
- uploadPartResponse,
- partNumber,
- uploadRequest.doRemoteDataIntegrityCheck()
- )
- );
+ CompletableFuture convertFuture = uploadPartResponseFuture.whenComplete((resp, throwable) -> {
+ try {
+ inputStream.close();
+ } catch (IOException ex) {
+ log.error(
+ () -> new ParameterizedMessage(
+ "Failed to close stream while uploading a part of idx {} and file {}.",
+ uploadPartRequest.partNumber(),
+ uploadPartRequest.key()
+ ),
+ ex
+ );
+ }
+ })
+ .thenApply(
+ uploadPartResponse -> convertUploadPartResponse(
+ completedParts,
+ inputStreamContainers,
+ uploadPartResponse,
+ partNumber,
+ uploadRequest.doRemoteDataIntegrityCheck()
+ )
+ );
futures.add(convertFuture);
CompletableFutureUtils.forwardExceptionTo(convertFuture, uploadPartResponseFuture);
diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java
index 8d45c2167a3d1..4f1ab9764702e 100644
--- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java
+++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java
@@ -35,9 +35,12 @@
import org.opensearch.common.util.ByteUtils;
import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.repositories.s3.SocketAccess;
+import org.opensearch.repositories.s3.StatsMetricPublisher;
import org.opensearch.repositories.s3.io.CheckedContainer;
+import java.io.BufferedInputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.util.Arrays;
import java.util.Base64;
import java.util.List;
@@ -58,6 +61,7 @@ public final class AsyncTransferManager {
private static final Logger log = LogManager.getLogger(AsyncTransferManager.class);
private final ExecutorService executorService;
private final ExecutorService priorityExecutorService;
+ private final ExecutorService urgentExecutorService;
private final long minimumPartSize;
/**
@@ -72,10 +76,16 @@ public final class AsyncTransferManager {
* @param executorService The stream reader {@link ExecutorService} for normal priority uploads
* @param priorityExecutorService The stream read {@link ExecutorService} for high priority uploads
*/
- public AsyncTransferManager(long minimumPartSize, ExecutorService executorService, ExecutorService priorityExecutorService) {
+ public AsyncTransferManager(
+ long minimumPartSize,
+ ExecutorService executorService,
+ ExecutorService priorityExecutorService,
+ ExecutorService urgentExecutorService
+ ) {
this.executorService = executorService;
this.priorityExecutorService = priorityExecutorService;
this.minimumPartSize = minimumPartSize;
+ this.urgentExecutorService = urgentExecutorService;
}
/**
@@ -86,16 +96,21 @@ public AsyncTransferManager(long minimumPartSize, ExecutorService executorServic
* @param streamContext The {@link StreamContext} to supply streams during upload
* @return A {@link CompletableFuture} to listen for upload completion
*/
- public CompletableFuture uploadObject(S3AsyncClient s3AsyncClient, UploadRequest uploadRequest, StreamContext streamContext) {
+ public CompletableFuture uploadObject(
+ S3AsyncClient s3AsyncClient,
+ UploadRequest uploadRequest,
+ StreamContext streamContext,
+ StatsMetricPublisher statsMetricPublisher
+ ) {
CompletableFuture returnFuture = new CompletableFuture<>();
try {
if (streamContext.getNumberOfParts() == 1) {
log.debug(() -> "Starting the upload as a single upload part request");
- uploadInOneChunk(s3AsyncClient, uploadRequest, streamContext.provideStream(0), returnFuture);
+ uploadInOneChunk(s3AsyncClient, uploadRequest, streamContext.provideStream(0), returnFuture, statsMetricPublisher);
} else {
log.debug(() -> "Starting the upload as multipart upload request");
- uploadInParts(s3AsyncClient, uploadRequest, streamContext, returnFuture);
+ uploadInParts(s3AsyncClient, uploadRequest, streamContext, returnFuture, statsMetricPublisher);
}
} catch (Throwable throwable) {
returnFuture.completeExceptionally(throwable);
@@ -108,12 +123,14 @@ private void uploadInParts(
S3AsyncClient s3AsyncClient,
UploadRequest uploadRequest,
StreamContext streamContext,
- CompletableFuture returnFuture
+ CompletableFuture returnFuture,
+ StatsMetricPublisher statsMetricPublisher
) {
CreateMultipartUploadRequest.Builder createMultipartUploadRequestBuilder = CreateMultipartUploadRequest.builder()
.bucket(uploadRequest.getBucket())
- .key(uploadRequest.getKey());
+ .key(uploadRequest.getKey())
+ .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector));
if (uploadRequest.doRemoteDataIntegrityCheck()) {
createMultipartUploadRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32);
}
@@ -152,6 +169,7 @@ private void doUploadInParts(
s3AsyncClient,
executorService,
priorityExecutorService,
+ urgentExecutorService,
uploadRequest,
streamContext,
uploadId,
@@ -286,28 +304,42 @@ private void uploadInOneChunk(
S3AsyncClient s3AsyncClient,
UploadRequest uploadRequest,
InputStreamContainer inputStreamContainer,
- CompletableFuture returnFuture
+ CompletableFuture returnFuture,
+ StatsMetricPublisher statsMetricPublisher
) {
PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder()
.bucket(uploadRequest.getBucket())
.key(uploadRequest.getKey())
- .contentLength(uploadRequest.getContentLength());
+ .contentLength(uploadRequest.getContentLength())
+ .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.putObjectMetricPublisher));
if (uploadRequest.doRemoteDataIntegrityCheck()) {
putObjectRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32);
putObjectRequestBuilder.checksumCRC32(base64StringFromLong(uploadRequest.getExpectedChecksum()));
}
- ExecutorService streamReadExecutor = uploadRequest.getWritePriority() == WritePriority.HIGH
- ? priorityExecutorService
- : executorService;
+ ExecutorService streamReadExecutor;
+ if (uploadRequest.getWritePriority() == WritePriority.URGENT) {
+ streamReadExecutor = urgentExecutorService;
+ } else if (uploadRequest.getWritePriority() == WritePriority.HIGH) {
+ streamReadExecutor = priorityExecutorService;
+ } else {
+ streamReadExecutor = executorService;
+ }
+ // Buffered stream is needed to allow mark and reset ops during IO errors so that only buffered
+ // data can be retried instead of retrying whole file by the application.
+ InputStream inputStream = new BufferedInputStream(inputStreamContainer.getInputStream(), (int) (ByteSizeUnit.MB.toBytes(1) + 1));
CompletableFuture putObjectFuture = SocketAccess.doPrivileged(
() -> s3AsyncClient.putObject(
putObjectRequestBuilder.build(),
- AsyncRequestBody.fromInputStream(
- inputStreamContainer.getInputStream(),
- inputStreamContainer.getContentLength(),
- streamReadExecutor
- )
+ AsyncRequestBody.fromInputStream(inputStream, inputStreamContainer.getContentLength(), streamReadExecutor)
).handle((resp, throwable) -> {
+ try {
+ inputStream.close();
+ } catch (IOException e) {
+ log.error(
+ () -> new ParameterizedMessage("Failed to close stream while uploading single file {}.", uploadRequest.getKey()),
+ e
+ );
+ }
if (throwable != null) {
Throwable unwrappedThrowable = ExceptionsHelper.unwrap(throwable, S3Exception.class);
if (unwrappedThrowable != null) {
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java
index a4bfe11383b4f..8e1926d40302f 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java
@@ -302,7 +302,7 @@ protected S3Repository createRepository(
ClusterService clusterService,
RecoverySettings recoverySettings
) {
- return new S3Repository(metadata, registry, service, clusterService, recoverySettings, null, null, null, null, false) {
+ return new S3Repository(metadata, registry, service, clusterService, recoverySettings, null, null, null, null, null, false) {
@Override
protected void assertSnapshotOrGenericThread() {
// eliminate thread name check as we create repo manually on test/main threads
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3AsyncServiceTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3AsyncServiceTests.java
index e9fe557ab751a..de9ad46bb222d 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3AsyncServiceTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3AsyncServiceTests.java
@@ -44,12 +44,12 @@ public void testCachedClientsAreReleased() {
final S3ClientSettings otherClientSettings = s3AsyncService.settings(metadata2);
assertSame(clientSettings, otherClientSettings);
final AmazonAsyncS3Reference reference = SocketAccess.doPrivileged(
- () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer)
+ () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer)
);
reference.close();
s3AsyncService.close();
final AmazonAsyncS3Reference referenceReloaded = SocketAccess.doPrivileged(
- () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer)
+ () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer)
);
assertNotSame(referenceReloaded, reference);
referenceReloaded.close();
@@ -79,12 +79,12 @@ public void testCachedClientsWithCredentialsAreReleased() {
final S3ClientSettings otherClientSettings = s3AsyncService.settings(metadata2);
assertSame(clientSettings, otherClientSettings);
final AmazonAsyncS3Reference reference = SocketAccess.doPrivileged(
- () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer)
+ () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer)
);
reference.close();
s3AsyncService.close();
final AmazonAsyncS3Reference referenceReloaded = SocketAccess.doPrivileged(
- () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer)
+ () -> s3AsyncService.client(metadata1, asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer)
);
assertNotSame(referenceReloaded, reference);
referenceReloaded.close();
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java
index 8c8524212e08e..7c67519f2f3b0 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java
@@ -64,6 +64,7 @@
import org.mockito.invocation.InvocationOnMock;
+import static org.opensearch.repositories.s3.S3Repository.BULK_DELETE_SIZE;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -265,10 +266,11 @@ public void verifySingleChunkUploadCallCount(boolean finalizeUploadFailure) {
@Override
public AmazonAsyncS3Reference client(
RepositoryMetadata repositoryMetadata,
+ AsyncExecutorContainer urgentExecutorBuilder,
AsyncExecutorContainer priorityExecutorBuilder,
AsyncExecutorContainer normalExecutorBuilder
) {
- return new AmazonAsyncS3Reference(AmazonAsyncS3WithCredentials.create(asyncClient, asyncClient, null));
+ return new AmazonAsyncS3Reference(AmazonAsyncS3WithCredentials.create(asyncClient, asyncClient, asyncClient, null));
}
}
@@ -387,13 +389,16 @@ private S3BlobStore createBlobStore() {
S3Repository.BUFFER_SIZE_SETTING.getDefault(Settings.EMPTY),
S3Repository.CANNED_ACL_SETTING.getDefault(Settings.EMPTY),
S3Repository.STORAGE_CLASS_SETTING.getDefault(Settings.EMPTY),
+ BULK_DELETE_SIZE.get(Settings.EMPTY),
repositoryMetadata,
new AsyncTransferManager(
S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(),
asyncExecutorContainer.getStreamReader(),
+ asyncExecutorContainer.getStreamReader(),
asyncExecutorContainer.getStreamReader()
),
asyncExecutorContainer,
+ asyncExecutorContainer,
asyncExecutorContainer
);
}
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java
index ecad68474b601..ceab06bd051e9 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java
@@ -95,6 +95,7 @@
import static org.opensearch.repositories.s3.S3ClientSettings.MAX_RETRIES_SETTING;
import static org.opensearch.repositories.s3.S3ClientSettings.READ_TIMEOUT_SETTING;
import static org.opensearch.repositories.s3.S3ClientSettings.REGION;
+import static org.opensearch.repositories.s3.S3Repository.BULK_DELETE_SIZE;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@@ -215,13 +216,16 @@ protected AsyncMultiStreamBlobContainer createBlobContainer(
bufferSize == null ? S3Repository.BUFFER_SIZE_SETTING.getDefault(Settings.EMPTY) : bufferSize,
S3Repository.CANNED_ACL_SETTING.getDefault(Settings.EMPTY),
S3Repository.STORAGE_CLASS_SETTING.getDefault(Settings.EMPTY),
+ BULK_DELETE_SIZE.get(Settings.EMPTY),
repositoryMetadata,
new AsyncTransferManager(
S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(),
asyncExecutorContainer.getStreamReader(),
+ asyncExecutorContainer.getStreamReader(),
asyncExecutorContainer.getStreamReader()
),
asyncExecutorContainer,
+ asyncExecutorContainer,
asyncExecutorContainer
)
) {
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java
index 9817d7cd520ef..58ad290a31e85 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java
@@ -276,10 +276,12 @@ public void testDelete() throws IOException {
final String bucketName = randomAlphaOfLengthBetween(1, 10);
final BlobPath blobPath = new BlobPath();
+ int bulkDeleteSize = 5;
final S3BlobStore blobStore = mock(S3BlobStore.class);
when(blobStore.bucket()).thenReturn(bucketName);
when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher());
+ when(blobStore.getBulkDeletesSize()).thenReturn(bulkDeleteSize);
final S3Client client = mock(S3Client.class);
doAnswer(invocation -> new AmazonS3Reference(client)).when(blobStore).clientReference();
@@ -297,8 +299,11 @@ public void testDelete() throws IOException {
when(client.listObjectsV2Paginator(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Iterable);
final List keysDeleted = new ArrayList<>();
+ AtomicInteger deleteCount = new AtomicInteger();
doAnswer(invocation -> {
DeleteObjectsRequest deleteObjectsRequest = invocation.getArgument(0);
+ deleteCount.getAndIncrement();
+ logger.info("Object sizes are{}", deleteObjectsRequest.delete().objects().size());
keysDeleted.addAll(deleteObjectsRequest.delete().objects().stream().map(ObjectIdentifier::key).collect(Collectors.toList()));
return DeleteObjectsResponse.builder().build();
}).when(client).deleteObjects(any(DeleteObjectsRequest.class));
@@ -311,6 +316,8 @@ public void testDelete() throws IOException {
// keysDeleted will have blobPath also
assertEquals(listObjectsV2ResponseIterator.getKeysListed().size(), keysDeleted.size() - 1);
assertTrue(keysDeleted.contains(blobPath.buildAsString()));
+ // keysDeleted will have blobPath also
+ assertEquals((int) Math.ceil(((double) keysDeleted.size() + 1) / bulkDeleteSize), deleteCount.get());
keysDeleted.remove(blobPath.buildAsString());
assertEquals(new HashSet<>(listObjectsV2ResponseIterator.getKeysListed()), new HashSet<>(keysDeleted));
}
@@ -928,7 +935,7 @@ public void testReadBlobAsyncMultiPart() throws Exception {
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
- AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, null)
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
);
final S3BlobStore blobStore = mock(S3BlobStore.class);
@@ -969,7 +976,7 @@ public void testReadBlobAsyncMultiPart() throws Exception {
assertEquals(objectSize, readContext.getBlobSize());
for (int partNumber = 1; partNumber < objectPartCount; partNumber++) {
- InputStreamContainer inputStreamContainer = readContext.getPartStreams().get(partNumber);
+ InputStreamContainer inputStreamContainer = readContext.getPartStreams().get(partNumber).get().join();
final int offset = partNumber * partSize;
assertEquals(partSize, inputStreamContainer.getContentLength());
assertEquals(offset, inputStreamContainer.getOffset());
@@ -986,7 +993,7 @@ public void testReadBlobAsyncSinglePart() throws Exception {
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
- AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, null)
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
);
final S3BlobStore blobStore = mock(S3BlobStore.class);
final BlobPath blobPath = new BlobPath();
@@ -1024,7 +1031,7 @@ public void testReadBlobAsyncSinglePart() throws Exception {
assertEquals(checksum, readContext.getBlobChecksum());
assertEquals(objectSize, readContext.getBlobSize());
- InputStreamContainer inputStreamContainer = readContext.getPartStreams().stream().findFirst().get();
+ InputStreamContainer inputStreamContainer = readContext.getPartStreams().stream().findFirst().get().get().join();
assertEquals(objectSize, inputStreamContainer.getContentLength());
assertEquals(0, inputStreamContainer.getOffset());
assertEquals(objectSize, inputStreamContainer.getInputStream().readAllBytes().length);
@@ -1041,7 +1048,7 @@ public void testReadBlobAsyncFailure() throws Exception {
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
- AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, null)
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
);
final S3BlobStore blobStore = mock(S3BlobStore.class);
@@ -1074,6 +1081,51 @@ public void testReadBlobAsyncFailure() throws Exception {
assertEquals(1, readContextActionListener.getFailureCount());
}
+ public void testReadBlobAsyncOnCompleteFailureMissingData() throws Exception {
+ final String bucketName = randomAlphaOfLengthBetween(1, 10);
+ final String blobName = randomAlphaOfLengthBetween(1, 10);
+ final String checksum = randomAlphaOfLength(10);
+
+ final long objectSize = 100L;
+ final int objectPartCount = 10;
+
+ final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
+ final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
+ );
+
+ final S3BlobStore blobStore = mock(S3BlobStore.class);
+ final BlobPath blobPath = new BlobPath();
+
+ when(blobStore.bucket()).thenReturn(bucketName);
+ when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher());
+ when(blobStore.serverSideEncryption()).thenReturn(false);
+ when(blobStore.asyncClientReference()).thenReturn(amazonAsyncS3Reference);
+
+ CompletableFuture getObjectAttributesResponseCompletableFuture = new CompletableFuture<>();
+ getObjectAttributesResponseCompletableFuture.complete(
+ GetObjectAttributesResponse.builder()
+ .checksum(Checksum.builder().build())
+ .objectSize(null)
+ .objectParts(GetObjectAttributesParts.builder().totalPartsCount(objectPartCount).build())
+ .build()
+ );
+ when(s3AsyncClient.getObjectAttributes(any(GetObjectAttributesRequest.class))).thenReturn(
+ getObjectAttributesResponseCompletableFuture
+ );
+
+ CountDownLatch countDownLatch = new CountDownLatch(1);
+ CountingCompletionListener readContextActionListener = new CountingCompletionListener<>();
+ LatchedActionListener listener = new LatchedActionListener<>(readContextActionListener, countDownLatch);
+
+ final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore);
+ blobContainer.readBlobAsync(blobName, listener);
+ countDownLatch.await();
+
+ assertEquals(0, readContextActionListener.getResponseCount());
+ assertEquals(1, readContextActionListener.getFailureCount());
+ }
+
public void testGetBlobMetadata() throws Exception {
final String checksum = randomAlphaOfLengthBetween(1, 10);
final long objectSize = 100L;
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
index 533c3aa17009d..6fec535ae6301 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java
@@ -36,17 +36,20 @@
import org.opensearch.cluster.metadata.RepositoryMetadata;
import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.core.common.unit.ByteSizeValue;
import org.opensearch.core.xcontent.NamedXContentRegistry;
import org.opensearch.indices.recovery.RecoverySettings;
import org.opensearch.repositories.RepositoryException;
+import org.opensearch.repositories.blobstore.BlobStoreRepository;
import org.opensearch.repositories.blobstore.BlobStoreTestUtil;
import org.opensearch.test.OpenSearchTestCase;
import org.hamcrest.Matchers;
import java.nio.file.Path;
+import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.containsString;
@@ -122,7 +125,8 @@ public void testBasePathSetting() {
}
public void testDefaultBufferSize() {
- final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", Settings.EMPTY);
+ Settings settings = Settings.builder().build();
+ final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", settings);
try (S3Repository s3repo = createS3Repo(metadata)) {
assertThat(s3repo.getBlobStore(), is(nullValue()));
s3repo.start();
@@ -133,6 +137,26 @@ public void testDefaultBufferSize() {
}
}
+ public void testIsReloadable() {
+ final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", Settings.EMPTY);
+ try (S3Repository s3repo = createS3Repo(metadata)) {
+ assertTrue(s3repo.isReloadable());
+ }
+ }
+
+ public void testRestrictedSettingsDefault() {
+ final RepositoryMetadata metadata = new RepositoryMetadata("dummy-repo", "mock", Settings.EMPTY);
+ try (S3Repository s3repo = createS3Repo(metadata)) {
+ List> restrictedSettings = s3repo.getRestrictedSystemRepositorySettings();
+ assertThat(restrictedSettings.size(), is(5));
+ assertTrue(restrictedSettings.contains(BlobStoreRepository.SYSTEM_REPOSITORY_SETTING));
+ assertTrue(restrictedSettings.contains(BlobStoreRepository.READONLY_SETTING));
+ assertTrue(restrictedSettings.contains(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY));
+ assertTrue(restrictedSettings.contains(S3Repository.BUCKET_SETTING));
+ assertTrue(restrictedSettings.contains(S3Repository.BASE_PATH_SETTING));
+ }
+ }
+
private S3Repository createS3Repo(RepositoryMetadata metadata) {
return new S3Repository(
metadata,
@@ -144,6 +168,7 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) {
null,
null,
null,
+ null,
false
) {
@Override
diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java
index 9c07b929052bc..2437547a80a6f 100644
--- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java
+++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java
@@ -33,12 +33,18 @@
import org.opensearch.common.io.InputStreamContainer;
import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.repositories.blobstore.ZeroInputStream;
+import org.opensearch.repositories.s3.StatsMetricPublisher;
import org.opensearch.test.OpenSearchTestCase;
import org.junit.Before;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
@@ -58,6 +64,7 @@ public void setUp() throws Exception {
asyncTransferManager = new AsyncTransferManager(
ByteSizeUnit.MB.toBytes(5),
Executors.newSingleThreadExecutor(),
+ Executors.newSingleThreadExecutor(),
Executors.newSingleThreadExecutor()
);
super.setUp();
@@ -70,17 +77,17 @@ public void testOneChunkUpload() {
putObjectResponseCompletableFuture
);
+ AtomicReference streamRef = new AtomicReference<>();
CompletableFuture resultFuture = asyncTransferManager.uploadObject(
s3AsyncClient,
new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(1), WritePriority.HIGH, uploadSuccess -> {
// do nothing
}, false, null),
- new StreamContext(
- (partIdx, partSize, position) -> new InputStreamContainer(new ZeroInputStream(partSize), partSize, position),
- ByteSizeUnit.MB.toBytes(1),
- ByteSizeUnit.MB.toBytes(1),
- 1
- )
+ new StreamContext((partIdx, partSize, position) -> {
+ streamRef.set(new ZeroInputStream(partSize));
+ return new InputStreamContainer(streamRef.get(), partSize, position);
+ }, ByteSizeUnit.MB.toBytes(1), ByteSizeUnit.MB.toBytes(1), 1),
+ new StatsMetricPublisher()
);
try {
@@ -90,6 +97,14 @@ public void testOneChunkUpload() {
}
verify(s3AsyncClient, times(1)).putObject(any(PutObjectRequest.class), any(AsyncRequestBody.class));
+
+ boolean closeError = false;
+ try {
+ streamRef.get().available();
+ } catch (IOException e) {
+ closeError = e.getMessage().equals("Stream closed");
+ }
+ assertTrue("InputStream was still open after upload", closeError);
}
public void testOneChunkUploadCorruption() {
@@ -118,7 +133,8 @@ public void testOneChunkUploadCorruption() {
ByteSizeUnit.MB.toBytes(1),
ByteSizeUnit.MB.toBytes(1),
1
- )
+ ),
+ new StatsMetricPublisher()
);
try {
@@ -159,17 +175,18 @@ public void testMultipartUpload() {
abortMultipartUploadResponseCompletableFuture
);
+ List streams = new ArrayList<>();
CompletableFuture resultFuture = asyncTransferManager.uploadObject(
s3AsyncClient,
new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(5), WritePriority.HIGH, uploadSuccess -> {
// do nothing
}, true, 3376132981L),
- new StreamContext(
- (partIdx, partSize, position) -> new InputStreamContainer(new ZeroInputStream(partSize), partSize, position),
- ByteSizeUnit.MB.toBytes(1),
- ByteSizeUnit.MB.toBytes(1),
- 5
- )
+ new StreamContext((partIdx, partSize, position) -> {
+ InputStream stream = new ZeroInputStream(partSize);
+ streams.add(stream);
+ return new InputStreamContainer(stream, partSize, position);
+ }, ByteSizeUnit.MB.toBytes(1), ByteSizeUnit.MB.toBytes(1), 5),
+ new StatsMetricPublisher()
);
try {
@@ -178,6 +195,16 @@ public void testMultipartUpload() {
fail("did not expect resultFuture to fail");
}
+ streams.forEach(stream -> {
+ boolean closeError = false;
+ try {
+ stream.available();
+ } catch (IOException e) {
+ closeError = e.getMessage().equals("Stream closed");
+ }
+ assertTrue("InputStream was still open after upload", closeError);
+ });
+
verify(s3AsyncClient, times(1)).createMultipartUpload(any(CreateMultipartUploadRequest.class));
verify(s3AsyncClient, times(5)).uploadPart(any(UploadPartRequest.class), any(AsyncRequestBody.class));
verify(s3AsyncClient, times(1)).completeMultipartUpload(any(CompleteMultipartUploadRequest.class));
@@ -219,7 +246,8 @@ public void testMultipartUploadCorruption() {
ByteSizeUnit.MB.toBytes(1),
ByteSizeUnit.MB.toBytes(1),
5
- )
+ ),
+ new StatsMetricPublisher()
);
try {
diff --git a/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbMMapDirectoryTests.java b/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbMMapDirectoryTests.java
index 2cac58262c75a..e1655cc5e0784 100644
--- a/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbMMapDirectoryTests.java
+++ b/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbMMapDirectoryTests.java
@@ -47,9 +47,9 @@ protected Directory getDirectory(Path file) throws IOException {
@Override
public void testCreateOutputForExistingFile() throws IOException {
- /**
- * This test is disabled because {@link SmbDirectoryWrapper} opens existing file
- * with an explicit StandardOpenOption.TRUNCATE_EXISTING option.
+ /*
+ This test is disabled because {@link SmbDirectoryWrapper} opens existing file
+ with an explicit StandardOpenOption.TRUNCATE_EXISTING option.
*/
}
}
diff --git a/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbNIOFSDirectoryTests.java b/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbNIOFSDirectoryTests.java
index 7390759029dfc..6f821147c3079 100644
--- a/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbNIOFSDirectoryTests.java
+++ b/plugins/store-smb/src/test/java/org/opensearch/index/store/SmbNIOFSDirectoryTests.java
@@ -26,9 +26,9 @@ protected Directory getDirectory(Path file) throws IOException {
@Override
public void testCreateOutputForExistingFile() throws IOException {
- /**
- * This test is disabled because {@link SmbDirectoryWrapper} opens existing file
- * with an explicit StandardOpenOption.TRUNCATE_EXISTING option.
+ /*
+ This test is disabled because {@link SmbDirectoryWrapper} opens existing file
+ with an explicit StandardOpenOption.TRUNCATE_EXISTING option.
*/
}
}
diff --git a/plugins/telemetry-otel/build.gradle b/plugins/telemetry-otel/build.gradle
index 45c9f522c09d8..f5c367cb7643b 100644
--- a/plugins/telemetry-otel/build.gradle
+++ b/plugins/telemetry-otel/build.gradle
@@ -28,7 +28,7 @@ dependencies {
api "io.opentelemetry:opentelemetry-sdk-trace:${versions.opentelemetry}"
api "io.opentelemetry:opentelemetry-sdk-metrics:${versions.opentelemetry}"
api "io.opentelemetry:opentelemetry-exporter-logging:${versions.opentelemetry}"
- api "io.opentelemetry:opentelemetry-semconv:${versions.opentelemetry}-alpha"
+ api "io.opentelemetry.semconv:opentelemetry-semconv:${versions.opentelemetrysemconv}"
api "io.opentelemetry:opentelemetry-sdk-logs:${versions.opentelemetry}"
api "io.opentelemetry:opentelemetry-exporter-otlp:${versions.opentelemetry}"
api "io.opentelemetry:opentelemetry-exporter-common:${versions.opentelemetry}"
@@ -37,6 +37,7 @@ dependencies {
runtimeOnly "com.squareup.okhttp3:okhttp:4.11.0"
runtimeOnly "com.squareup.okio:okio-jvm:3.5.0"
runtimeOnly "io.opentelemetry:opentelemetry-exporter-sender-okhttp:${versions.opentelemetry}"
+ api "io.opentelemetry:opentelemetry-extension-incubator:${versions.opentelemetry}-alpha"
testImplementation "io.opentelemetry:opentelemetry-sdk-testing:${versions.opentelemetry}"
}
@@ -80,29 +81,12 @@ thirdPartyAudit {
'io.opentelemetry.api.events.EventEmitter',
'io.opentelemetry.api.events.EventEmitterBuilder',
'io.opentelemetry.api.events.EventEmitterProvider',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedLongHistogramBuilder',
'io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties',
'io.opentelemetry.sdk.autoconfigure.spi.logs.ConfigurableLogRecordExporterProvider',
'io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider',
'io.opentelemetry.sdk.autoconfigure.spi.traces.ConfigurableSpanExporterProvider',
- 'io.opentelemetry.extension.incubator.metrics.DoubleCounterAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.DoubleGauge',
- 'io.opentelemetry.extension.incubator.metrics.DoubleGaugeAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.DoubleHistogramAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.DoubleUpDownCounterAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedDoubleCounterBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedDoubleGaugeBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedDoubleUpDownCounterBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedLongGaugeBuilder',
- 'io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder',
- 'io.opentelemetry.extension.incubator.metrics.LongCounterAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.LongGauge',
- 'io.opentelemetry.extension.incubator.metrics.LongGaugeAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.LongHistogramAdviceConfigurer',
- 'io.opentelemetry.extension.incubator.metrics.LongUpDownCounterAdviceConfigurer',
- 'kotlin.io.path.PathsKt'
+ 'kotlin.io.path.PathsKt',
+ 'io.opentelemetry.sdk.autoconfigure.spi.traces.ConfigurableSpanExporterProvider'
)
}
diff --git a/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties b/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties
index 544f42bd5513b..8dec1119eec66 100644
--- a/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties
+++ b/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties
@@ -25,3 +25,23 @@ logger.exporter.name = io.opentelemetry.exporter.logging.LoggingSpanExporter
logger.exporter.level = INFO
logger.exporter.appenderRef.tracing.ref = tracing
logger.exporter.additivity = false
+
+
+appender.metrics.type = RollingFile
+appender.metrics.name = metrics
+appender.metrics.fileName = ${sys:opensearch.logs.base_path}${sys:file.separator}${sys:opensearch.logs.cluster_name}_otel_metrics.log
+appender.metrics.filePermissions = rw-r-----
+appender.metrics.layout.type = PatternLayout
+appender.metrics.layout.pattern = %m%n
+appender.metrics.filePattern = ${sys:opensearch.logs.base_path}${sys:file.separator}${sys:opensearch.logs.cluster_name}_otel_metrics-%i.log.gz
+appender.metrics.policies.type = Policies
+appender.metrics.policies.size.type = SizeBasedTriggeringPolicy
+appender.metrics.policies.size.size = 1GB
+appender.metrics.strategy.type = DefaultRolloverStrategy
+appender.metrics.strategy.max = 4
+
+
+logger.metrics_exporter.name = io.opentelemetry.exporter.logging.LoggingMetricExporter
+logger.metrics_exporter.level = INFO
+logger.metrics_exporter.appenderRef.tracing.ref = metrics
+logger.metrics_exporter.additivity = false
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-1.30.1.jar.sha1
deleted file mode 100644
index b0ce00e191830..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-api-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-a32dfbd7f01de6711fd0e970f8d4b4c0405056d6
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..eae141a8d1a23
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-api-1.31.0.jar.sha1
@@ -0,0 +1 @@
+bb24a44d73484c681c236aed84fe6c28d17f30e2
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-context-1.30.1.jar.sha1
deleted file mode 100644
index 84cb60a2f7acb..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-context-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-58f665ff01ce6b964cdf0b8cb5cd1c196dfe94ce
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-context-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..6e42973adc581
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-context-1.31.0.jar.sha1
@@ -0,0 +1 @@
+b8004737f7a970124e36ac71fde8eb88423e8cee
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.30.1.jar.sha1
deleted file mode 100644
index eccb15f7b7c8e..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-f299d336dba1039478497f37b273dfa764c6faef
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..b119468e7f88b
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.31.0.jar.sha1
@@ -0,0 +1 @@
+b7b4baf5f9af72d5eb8a231dfb114ae31c57150d
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.30.1.jar.sha1
deleted file mode 100644
index 40537a399ab14..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-58f1a09e89955e6145babf8bcdf80c95174eb817
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..8f653922d6418
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.31.0.jar.sha1
@@ -0,0 +1 @@
+260e5363dad83a0ae65c16ad6a3dd2914e0db201
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.30.1.jar.sha1
deleted file mode 100644
index e88b7514ee54d..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-15692246539571c41180aff2b55abe527b939a7b
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..103da4720de96
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.31.0.jar.sha1
@@ -0,0 +1 @@
+b6454464425dfd81519070caeca3824558a2f1ae
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.30.1.jar.sha1
deleted file mode 100644
index 86937743208c6..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-947cf43a6411c4a323e14594431040a476ad43e8
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..3db07532ceea9
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.31.0.jar.sha1
@@ -0,0 +1 @@
+d8c22b6851bbc3dbf5d2387b9bde158ed5416ba4
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.30.1.jar.sha1
deleted file mode 100644
index 068926277253c..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-9f3a14515500e4df260ce7b10a668237a95ac791
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..10d9b7cdfe3e3
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.31.0.jar.sha1
@@ -0,0 +1 @@
+dd209381d58cfe81a989e29c9ca26d97c8dabd7a
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.31.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.31.0-alpha.jar.sha1
new file mode 100644
index 0000000000000..162890965a6eb
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.31.0-alpha.jar.sha1
@@ -0,0 +1 @@
+6c9f5c063309d92b6dd28bff0667f54b63afd36f
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-LICENSE.txt
new file mode 100644
index 0000000000000..d645695673349
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/plugins/crypto-kms/licenses/reactive-streams-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt
similarity index 100%
rename from plugins/crypto-kms/licenses/reactive-streams-NOTICE.txt
rename to plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.30.1.jar.sha1
deleted file mode 100644
index d425ed61cc4cd..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-4d15a9ea26e8e6ea93287a9f4ee02d91e5a74392
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..d6ce31a31cc6f
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.31.0.jar.sha1
@@ -0,0 +1 @@
+2b2093be08a09ac536292bf6cecf8129cc7fb191
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.30.1.jar.sha1
deleted file mode 100644
index 6b32d98b0f7c7..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8e437ba87004bb63069d04fb06beae65b98dd13a
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..8a6a9705d836d
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.31.0.jar.sha1
@@ -0,0 +1 @@
+f492528288236e097e12fc1c45963dd82c70d33c
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.30.1.jar.sha1
deleted file mode 100644
index 13ef6de11e82d..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-5985d0950746ad12b49cc42c063f26ddfbcaaacb
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..37d79f5c573f7
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.31.0.jar.sha1
@@ -0,0 +1 @@
+a63a203d3dc6f8875f8c26b9e3b522dc9a3f6280
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.30.1.jar.sha1
deleted file mode 100644
index fc5aad9c9011e..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-b12825541c5dae52a0fb35045c1b36df3ca8f632
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..80179e4808f50
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.31.0.jar.sha1
@@ -0,0 +1 @@
+47cc23762fae728d68e4fda1dfb71986ae0b8b3e
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.30.1.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.30.1.jar.sha1
deleted file mode 100644
index ac522b765da05..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.30.1.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-4c5531fbc44178a7bcfeb7021ae80e70a7c43458
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.31.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.31.0.jar.sha1
new file mode 100644
index 0000000000000..fd917a58ba77c
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.31.0.jar.sha1
@@ -0,0 +1 @@
+a3941197cfb8ae9eb9e482073480c0c3918b746c
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.21.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.21.0-alpha.jar.sha1
new file mode 100644
index 0000000000000..77b12c99464f6
--- /dev/null
+++ b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.21.0-alpha.jar.sha1
@@ -0,0 +1 @@
+207660e74d1e155272e9559fd4d27854b92fc6ac
\ No newline at end of file
diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.30.1-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.30.1-alpha.jar.sha1
deleted file mode 100644
index 089a2484dd1d5..0000000000000
--- a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.30.1-alpha.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8e8f7a97a4896a81846553275b9d61885be7ef50
\ No newline at end of file
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/IntegrationTestOTelTelemetryPlugin.java
similarity index 85%
rename from plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java
rename to plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/IntegrationTestOTelTelemetryPlugin.java
index ed4d13f3abb7d..45caf8bf5f60b 100644
--- a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/IntegrationTestOTelTelemetryPlugin.java
@@ -6,12 +6,9 @@
* compatible open source license.
*/
-package org.opensearch.telemetry.tracing;
+package org.opensearch.telemetry;
import org.opensearch.common.settings.Settings;
-import org.opensearch.telemetry.OTelTelemetryPlugin;
-import org.opensearch.telemetry.Telemetry;
-import org.opensearch.telemetry.TelemetrySettings;
import java.util.Optional;
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/InMemorySingletonMetricsExporter.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/InMemorySingletonMetricsExporter.java
new file mode 100644
index 0000000000000..74fc872cb30e3
--- /dev/null
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/InMemorySingletonMetricsExporter.java
@@ -0,0 +1,65 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import java.util.Collection;
+import java.util.List;
+
+import io.opentelemetry.sdk.common.CompletableResultCode;
+import io.opentelemetry.sdk.metrics.InstrumentType;
+import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
+import io.opentelemetry.sdk.metrics.data.MetricData;
+import io.opentelemetry.sdk.metrics.export.MetricExporter;
+import io.opentelemetry.sdk.testing.exporter.InMemoryMetricExporter;
+
+public class InMemorySingletonMetricsExporter implements MetricExporter {
+
+ public static final InMemorySingletonMetricsExporter INSTANCE = new InMemorySingletonMetricsExporter(InMemoryMetricExporter.create());
+
+ private static InMemoryMetricExporter delegate;
+
+ public static InMemorySingletonMetricsExporter create() {
+ return INSTANCE;
+ }
+
+ private InMemorySingletonMetricsExporter(InMemoryMetricExporter delegate) {
+ InMemorySingletonMetricsExporter.delegate = delegate;
+ }
+
+ @Override
+ public CompletableResultCode export(Collection metrics) {
+ return delegate.export(metrics);
+ }
+
+ @Override
+ public CompletableResultCode flush() {
+ return delegate.flush();
+ }
+
+ @Override
+ public CompletableResultCode shutdown() {
+ return delegate.shutdown();
+ }
+
+ public List getFinishedMetricItems() {
+ return delegate.getFinishedMetricItems();
+ }
+
+ /**
+ * Clears the state.
+ */
+ public void reset() {
+ delegate.reset();
+ }
+
+ @Override
+ public AggregationTemporality getAggregationTemporality(InstrumentType instrumentType) {
+ return delegate.getAggregationTemporality(instrumentType);
+ }
+}
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsDisabledSanityIT.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsDisabledSanityIT.java
new file mode 100644
index 0000000000000..bcdcb657c4f42
--- /dev/null
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsDisabledSanityIT.java
@@ -0,0 +1,62 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.telemetry.IntegrationTestOTelTelemetryPlugin;
+import org.opensearch.telemetry.OTelTelemetrySettings;
+import org.opensearch.telemetry.TelemetrySettings;
+import org.opensearch.telemetry.metrics.noop.NoopCounter;
+import org.opensearch.telemetry.metrics.noop.NoopMetricsRegistry;
+import org.opensearch.test.OpenSearchIntegTestCase;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, minNumDataNodes = 1)
+public class TelemetryMetricsDisabledSanityIT extends OpenSearchIntegTestCase {
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put(TelemetrySettings.METRICS_FEATURE_ENABLED_SETTING.getKey(), false)
+ .put(
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(),
+ "org.opensearch.telemetry.metrics.InMemorySingletonMetricsExporter"
+ )
+ .put(TelemetrySettings.METRICS_PUBLISH_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(1))
+ .build();
+ }
+
+ @Override
+ protected Collection> nodePlugins() {
+ return Arrays.asList(IntegrationTestOTelTelemetryPlugin.class);
+ }
+
+ @Override
+ protected boolean addMockTelemetryPlugin() {
+ return false;
+ }
+
+ public void testSanityChecksWhenMetricsDisabled() throws Exception {
+ MetricsRegistry metricsRegistry = internalCluster().getInstance(MetricsRegistry.class);
+
+ Counter counter = metricsRegistry.createCounter("test-counter", "test", "1");
+ counter.add(1.0);
+
+ Thread.sleep(2000);
+
+ assertTrue(metricsRegistry instanceof NoopMetricsRegistry);
+ assertTrue(counter instanceof NoopCounter);
+ }
+
+}
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java
new file mode 100644
index 0000000000000..ed341595d327d
--- /dev/null
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/metrics/TelemetryMetricsEnabledSanityIT.java
@@ -0,0 +1,99 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.telemetry.IntegrationTestOTelTelemetryPlugin;
+import org.opensearch.telemetry.OTelTelemetrySettings;
+import org.opensearch.telemetry.TelemetrySettings;
+import org.opensearch.test.OpenSearchIntegTestCase;
+import org.junit.After;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import io.opentelemetry.sdk.metrics.data.DoublePointData;
+
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.SUITE, minNumDataNodes = 1)
+public class TelemetryMetricsEnabledSanityIT extends OpenSearchIntegTestCase {
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put(TelemetrySettings.METRICS_FEATURE_ENABLED_SETTING.getKey(), true)
+ .put(
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(),
+ "org.opensearch.telemetry.metrics.InMemorySingletonMetricsExporter"
+ )
+ .put(TelemetrySettings.METRICS_PUBLISH_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(1))
+ .build();
+ }
+
+ @Override
+ protected Collection> nodePlugins() {
+ return Arrays.asList(IntegrationTestOTelTelemetryPlugin.class);
+ }
+
+ @Override
+ protected boolean addMockTelemetryPlugin() {
+ return false;
+ }
+
+ public void testCounter() throws Exception {
+ MetricsRegistry metricsRegistry = internalCluster().getInstance(MetricsRegistry.class);
+ InMemorySingletonMetricsExporter.INSTANCE.reset();
+
+ Counter counter = metricsRegistry.createCounter("test-counter", "test", "1");
+ counter.add(1.0);
+ // Sleep for about 2s to wait for metrics to be published.
+ Thread.sleep(2000);
+
+ InMemorySingletonMetricsExporter exporter = InMemorySingletonMetricsExporter.INSTANCE;
+ double value = ((DoublePointData) ((ArrayList) exporter.getFinishedMetricItems()
+ .stream()
+ .filter(a -> a.getName().equals("test-counter"))
+ .collect(Collectors.toList())
+ .get(0)
+ .getDoubleSumData()
+ .getPoints()).get(0)).getValue();
+ assertEquals(1.0, value, 0.0);
+ }
+
+ public void testUpDownCounter() throws Exception {
+
+ MetricsRegistry metricsRegistry = internalCluster().getInstance(MetricsRegistry.class);
+ InMemorySingletonMetricsExporter.INSTANCE.reset();
+
+ Counter counter = metricsRegistry.createUpDownCounter("test-up-down-counter", "test", "1");
+ counter.add(1.0);
+ counter.add(-2.0);
+ // Sleep for about 2s to wait for metrics to be published.
+ Thread.sleep(2000);
+
+ InMemorySingletonMetricsExporter exporter = InMemorySingletonMetricsExporter.INSTANCE;
+ double value = ((DoublePointData) ((ArrayList) exporter.getFinishedMetricItems()
+ .stream()
+ .filter(a -> a.getName().equals("test-up-down-counter"))
+ .collect(Collectors.toList())
+ .get(0)
+ .getDoubleSumData()
+ .getPoints()).get(0)).getValue();
+ assertEquals(-1.0, value, 0.0);
+ }
+
+ @After
+ public void reset() {
+ InMemorySingletonMetricsExporter.INSTANCE.reset();
+ }
+}
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerDisabledSanityIT.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerDisabledSanityIT.java
index 949a58f6cab41..45ed140e1be94 100644
--- a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerDisabledSanityIT.java
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerDisabledSanityIT.java
@@ -12,6 +12,7 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.plugins.Plugin;
+import org.opensearch.telemetry.IntegrationTestOTelTelemetryPlugin;
import org.opensearch.telemetry.OTelTelemetrySettings;
import org.opensearch.telemetry.TelemetrySettings;
import org.opensearch.test.OpenSearchIntegTestCase;
diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerEnabledSanityIT.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerEnabledSanityIT.java
index 8a49a0abf5512..f07f2b308e801 100644
--- a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerEnabledSanityIT.java
+++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/TelemetryTracerEnabledSanityIT.java
@@ -12,6 +12,7 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.plugins.Plugin;
+import org.opensearch.telemetry.IntegrationTestOTelTelemetryPlugin;
import org.opensearch.telemetry.OTelTelemetrySettings;
import org.opensearch.telemetry.TelemetrySettings;
import org.opensearch.telemetry.tracing.attributes.Attributes;
@@ -88,9 +89,7 @@ public void testSanityChecksWhenTracingEnabled() throws Exception {
);
InMemorySingletonSpanExporter exporter = InMemorySingletonSpanExporter.INSTANCE;
- if (!exporter.getFinishedSpanItems().isEmpty()) {
- validators.validate(exporter.getFinishedSpanItems(), 6);
- }
+ validators.validate(exporter.getFinishedSpanItems(), 6);
}
private static void updateTelemetrySetting(Client client, boolean value) {
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelAttributesConverter.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelAttributesConverter.java
similarity index 71%
rename from plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelAttributesConverter.java
rename to plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelAttributesConverter.java
index 4d0966e6b5185..98d265e92ba3c 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelAttributesConverter.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelAttributesConverter.java
@@ -6,7 +6,9 @@
* compatible open source license.
*/
-package org.opensearch.telemetry.tracing;
+package org.opensearch.telemetry;
+
+import org.opensearch.telemetry.metrics.tags.Tags;
import java.util.Locale;
@@ -16,7 +18,7 @@
/**
* Converts {@link org.opensearch.telemetry.tracing.attributes.Attributes} to OTel {@link Attributes}
*/
-final class OTelAttributesConverter {
+public final class OTelAttributesConverter {
/**
* Constructor.
@@ -28,7 +30,7 @@ private OTelAttributesConverter() {}
* @param attributes attributes
* @return otel attributes.
*/
- static Attributes convert(org.opensearch.telemetry.tracing.attributes.Attributes attributes) {
+ public static Attributes convert(org.opensearch.telemetry.tracing.attributes.Attributes attributes) {
AttributesBuilder attributesBuilder = Attributes.builder();
if (attributes != null) {
attributes.getAttributesMap().forEach((x, y) -> addSpanAttribute(x, y, attributesBuilder));
@@ -49,4 +51,17 @@ private static void addSpanAttribute(String key, Object value, AttributesBuilder
throw new IllegalArgumentException(String.format(Locale.ROOT, "Span attribute value %s type not supported", value));
}
}
+
+ /**
+ * Attribute converter.
+ * @param tags attributes
+ * @return otel attributes.
+ */
+ public static Attributes convert(Tags tags) {
+ AttributesBuilder attributesBuilder = Attributes.builder();
+ if (tags != null) {
+ tags.getTagsMap().forEach((x, y) -> addSpanAttribute(x, y, attributesBuilder));
+ }
+ return attributesBuilder.build();
+ }
}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java
index 1af88196e3727..297ae8873636f 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java
@@ -8,28 +8,36 @@
package org.opensearch.telemetry;
+import org.opensearch.common.concurrent.RefCountedReleasable;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.plugins.Plugin;
import org.opensearch.plugins.TelemetryPlugin;
-import org.opensearch.telemetry.metrics.MetricsTelemetry;
import org.opensearch.telemetry.tracing.OTelResourceProvider;
import org.opensearch.telemetry.tracing.OTelTelemetry;
-import org.opensearch.telemetry.tracing.OTelTracingTelemetry;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
+import io.opentelemetry.sdk.OpenTelemetrySdk;
+
/**
* Telemetry plugin based on Otel
*/
public class OTelTelemetryPlugin extends Plugin implements TelemetryPlugin {
+ /**
+ * Instrumentation scope name.
+ */
+ public static final String INSTRUMENTATION_SCOPE_NAME = "org.opensearch.telemetry";
+
static final String OTEL_TRACER_NAME = "otel";
private final Settings settings;
+ private RefCountedReleasable refCountedOpenTelemetry;
+
/**
* Creates Otel plugin
* @param settings cluster settings
@@ -44,23 +52,39 @@ public List> getSettings() {
OTelTelemetrySettings.TRACER_EXPORTER_BATCH_SIZE_SETTING,
OTelTelemetrySettings.TRACER_EXPORTER_DELAY_SETTING,
OTelTelemetrySettings.TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING,
- OTelTelemetrySettings.OTEL_TRACER_SPAN_EXPORTER_CLASS_SETTING
+ OTelTelemetrySettings.OTEL_TRACER_SPAN_EXPORTER_CLASS_SETTING,
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING
);
}
@Override
public Optional getTelemetry(TelemetrySettings telemetrySettings) {
+ initializeOpenTelemetrySdk(telemetrySettings);
return Optional.of(telemetry(telemetrySettings));
}
+ private void initializeOpenTelemetrySdk(TelemetrySettings telemetrySettings) {
+ if (refCountedOpenTelemetry != null) {
+ return;
+ }
+ OpenTelemetrySdk openTelemetrySdk = OTelResourceProvider.get(telemetrySettings, settings);
+ refCountedOpenTelemetry = new RefCountedReleasable<>("openTelemetry", openTelemetrySdk, openTelemetrySdk::close);
+ }
+
@Override
public String getName() {
return OTEL_TRACER_NAME;
}
private Telemetry telemetry(TelemetrySettings telemetrySettings) {
- return new OTelTelemetry(new OTelTracingTelemetry(OTelResourceProvider.get(telemetrySettings, settings)), new MetricsTelemetry() {
- });
+ return new OTelTelemetry(refCountedOpenTelemetry);
+ }
+
+ @Override
+ public void close() {
+ if (refCountedOpenTelemetry != null) {
+ refCountedOpenTelemetry.close();
+ }
}
}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetrySettings.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetrySettings.java
index 59c87cca22986..8e23f724b4570 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetrySettings.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetrySettings.java
@@ -11,13 +11,16 @@
import org.opensearch.SpecialPermission;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.unit.TimeValue;
+import org.opensearch.telemetry.metrics.exporter.OTelMetricsExporterFactory;
import org.opensearch.telemetry.tracing.exporter.OTelSpanExporterFactory;
import java.security.AccessController;
import java.security.PrivilegedActionException;
import java.security.PrivilegedExceptionAction;
+import io.opentelemetry.exporter.logging.LoggingMetricExporter;
import io.opentelemetry.exporter.logging.LoggingSpanExporter;
+import io.opentelemetry.sdk.metrics.export.MetricExporter;
import io.opentelemetry.sdk.trace.export.SpanExporter;
/**
@@ -83,4 +86,28 @@ private OTelTelemetrySettings() {}
Setting.Property.NodeScope,
Setting.Property.Final
);
+
+ /**
+ * Metrics Exporter type setting.
+ */
+ @SuppressWarnings("unchecked")
+ public static final Setting> OTEL_METRICS_EXPORTER_CLASS_SETTING = new Setting<>(
+ "telemetry.otel.metrics.exporter.class",
+ LoggingMetricExporter.class.getName(),
+ className -> {
+ // Check we ourselves are not being called by unprivileged code.
+ SpecialPermission.check();
+
+ try {
+ return AccessController.doPrivileged((PrivilegedExceptionAction>) () -> {
+ final ClassLoader loader = OTelMetricsExporterFactory.class.getClassLoader();
+ return (Class) loader.loadClass(className);
+ });
+ } catch (PrivilegedActionException ex) {
+ throw new IllegalStateException("Unable to load span exporter class:" + className, ex.getCause());
+ }
+ },
+ Setting.Property.NodeScope,
+ Setting.Property.Final
+ );
}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelCounter.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelCounter.java
new file mode 100644
index 0000000000000..b72f63e027243
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelCounter.java
@@ -0,0 +1,40 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.telemetry.OTelAttributesConverter;
+import org.opensearch.telemetry.metrics.tags.Tags;
+
+import io.opentelemetry.api.metrics.DoubleCounter;
+
+/**
+ * OTel Counter
+ */
+class OTelCounter implements Counter {
+
+ private final DoubleCounter otelDoubleCounter;
+
+ /**
+ * Constructor
+ * @param otelDoubleCounter delegate counter.
+ */
+ public OTelCounter(DoubleCounter otelDoubleCounter) {
+ this.otelDoubleCounter = otelDoubleCounter;
+ }
+
+ @Override
+ public void add(double value) {
+ otelDoubleCounter.add(value);
+ }
+
+ @Override
+ public void add(double value, Tags tags) {
+ otelDoubleCounter.add(value, OTelAttributesConverter.convert(tags));
+ }
+}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetry.java
new file mode 100644
index 0000000000000..6160e5106c041
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetry.java
@@ -0,0 +1,74 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.common.concurrent.RefCountedReleasable;
+import org.opensearch.telemetry.OTelTelemetryPlugin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+import io.opentelemetry.api.metrics.DoubleCounter;
+import io.opentelemetry.api.metrics.DoubleUpDownCounter;
+import io.opentelemetry.api.metrics.Meter;
+import io.opentelemetry.api.metrics.MeterProvider;
+import io.opentelemetry.sdk.OpenTelemetrySdk;
+
+/**
+ * OTel implementation for {@link MetricsTelemetry}
+ */
+public class OTelMetricsTelemetry implements MetricsTelemetry {
+ private final RefCountedReleasable refCountedOpenTelemetry;
+ private final Meter otelMeter;
+ private final T meterProvider;
+
+ /**
+ * Creates OTel based {@link MetricsTelemetry}.
+ * @param openTelemetry open telemetry.
+ * @param meterProvider {@link MeterProvider} instance
+ */
+ public OTelMetricsTelemetry(RefCountedReleasable openTelemetry, T meterProvider) {
+ this.refCountedOpenTelemetry = openTelemetry;
+ this.refCountedOpenTelemetry.incRef();
+ this.meterProvider = meterProvider;
+ this.otelMeter = meterProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME);
+ }
+
+ @Override
+ public Counter createCounter(String name, String description, String unit) {
+ DoubleCounter doubleCounter = AccessController.doPrivileged(
+ (PrivilegedAction) () -> otelMeter.counterBuilder(name)
+ .setUnit(unit)
+ .setDescription(description)
+ .ofDoubles()
+ .build()
+ );
+ return new OTelCounter(doubleCounter);
+ }
+
+ @Override
+ public Counter createUpDownCounter(String name, String description, String unit) {
+ DoubleUpDownCounter doubleUpDownCounter = AccessController.doPrivileged(
+ (PrivilegedAction) () -> otelMeter.upDownCounterBuilder(name)
+ .setUnit(unit)
+ .setDescription(description)
+ .ofDoubles()
+ .build()
+ );
+ return new OTelUpDownCounter(doubleUpDownCounter);
+ }
+
+ @Override
+ public void close() throws IOException {
+ meterProvider.close();
+ refCountedOpenTelemetry.close();
+ }
+}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelUpDownCounter.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelUpDownCounter.java
new file mode 100644
index 0000000000000..2f40881996f7e
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/OTelUpDownCounter.java
@@ -0,0 +1,40 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.telemetry.OTelAttributesConverter;
+import org.opensearch.telemetry.metrics.tags.Tags;
+
+import io.opentelemetry.api.metrics.DoubleUpDownCounter;
+
+/**
+ * OTel Counter
+ */
+public class OTelUpDownCounter implements Counter {
+
+ private final DoubleUpDownCounter doubleUpDownCounter;
+
+ /**
+ * Constructor
+ * @param doubleUpDownCounter delegate counter.
+ */
+ public OTelUpDownCounter(DoubleUpDownCounter doubleUpDownCounter) {
+ this.doubleUpDownCounter = doubleUpDownCounter;
+ }
+
+ @Override
+ public void add(double value) {
+ doubleUpDownCounter.add(value);
+ }
+
+ @Override
+ public void add(double value, Tags tags) {
+ doubleUpDownCounter.add(value, OTelAttributesConverter.convert(tags));
+ }
+}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactory.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactory.java
new file mode 100644
index 0000000000000..ef5a31e4003ca
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactory.java
@@ -0,0 +1,90 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics.exporter;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.opensearch.SpecialPermission;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.telemetry.OTelTelemetrySettings;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.lang.reflect.Method;
+import java.security.AccessController;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
+import io.opentelemetry.sdk.metrics.export.MetricExporter;
+
+/**
+ * Factory class to create the {@link MetricExporter} instance.
+ */
+public class OTelMetricsExporterFactory {
+
+ private static final Logger logger = LogManager.getLogger(OTelMetricsExporterFactory.class);
+
+ /**
+ * Base constructor.
+ */
+ private OTelMetricsExporterFactory() {
+
+ }
+
+ /**
+ * Creates the {@link MetricExporter} instances based on the OTEL_METRIC_EXPORTER_CLASS_SETTING value.
+ * As of now, it expects the MetricExporter implementations to have a create factory method to instantiate the
+ * MetricExporter.
+ * @param settings settings.
+ * @return MetricExporter instance.
+ */
+ public static MetricExporter create(Settings settings) {
+ Class MetricExporterProviderClass = OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.get(settings);
+ MetricExporter metricExporter = instantiateExporter(MetricExporterProviderClass);
+ logger.info("Successfully instantiated the Metrics MetricExporter class {}", MetricExporterProviderClass);
+ return metricExporter;
+ }
+
+ private static MetricExporter instantiateExporter(Class exporterProviderClass) {
+ try {
+ // Check we ourselves are not being called by unprivileged code.
+ SpecialPermission.check();
+ return AccessController.doPrivileged((PrivilegedExceptionAction) () -> {
+ String methodName = "create";
+ String getDefaultMethod = "getDefault";
+ for (Method m : exporterProviderClass.getMethods()) {
+ if (m.getName().equals(getDefaultMethod)) {
+ methodName = getDefaultMethod;
+ break;
+ }
+ }
+ try {
+ return (MetricExporter) MethodHandles.publicLookup()
+ .findStatic(exporterProviderClass, methodName, MethodType.methodType(exporterProviderClass))
+ .asType(MethodType.methodType(MetricExporter.class))
+ .invokeExact();
+ } catch (Throwable e) {
+ if (e.getCause() instanceof NoSuchMethodException) {
+ throw new IllegalStateException("No create factory method exist in [" + exporterProviderClass.getName() + "]");
+ } else {
+ throw new IllegalStateException(
+ "MetricExporter instantiation failed for class [" + exporterProviderClass.getName() + "]",
+ e.getCause()
+ );
+ }
+ }
+ });
+ } catch (PrivilegedActionException ex) {
+ throw new IllegalStateException(
+ "MetricExporter instantiation failed for class [" + exporterProviderClass.getName() + "]",
+ ex.getCause()
+ );
+ }
+ }
+}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/package-info.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/package-info.java
new file mode 100644
index 0000000000000..b48ec3e2336c4
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/exporter/package-info.java
@@ -0,0 +1,12 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/**
+ * This package contains classes needed for tracing requests.
+ */
+package org.opensearch.telemetry.metrics.exporter;
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/package-info.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/package-info.java
new file mode 100644
index 0000000000000..803c159eb201a
--- /dev/null
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/metrics/package-info.java
@@ -0,0 +1,12 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/**
+ * This package contains classes needed for tracing requests.
+ */
+package org.opensearch.telemetry.metrics;
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java
index fe05cc8bb7a41..14a19f122c17b 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java
@@ -10,6 +10,7 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.telemetry.TelemetrySettings;
+import org.opensearch.telemetry.metrics.exporter.OTelMetricsExporterFactory;
import org.opensearch.telemetry.tracing.exporter.OTelSpanExporterFactory;
import org.opensearch.telemetry.tracing.sampler.ProbabilisticSampler;
import org.opensearch.telemetry.tracing.sampler.RequestSampler;
@@ -18,17 +19,18 @@
import java.security.PrivilegedAction;
import java.util.concurrent.TimeUnit;
-import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.api.common.Attributes;
import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator;
import io.opentelemetry.context.propagation.ContextPropagators;
import io.opentelemetry.sdk.OpenTelemetrySdk;
+import io.opentelemetry.sdk.metrics.SdkMeterProvider;
+import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader;
import io.opentelemetry.sdk.resources.Resource;
import io.opentelemetry.sdk.trace.SdkTracerProvider;
import io.opentelemetry.sdk.trace.export.BatchSpanProcessor;
import io.opentelemetry.sdk.trace.export.SpanExporter;
import io.opentelemetry.sdk.trace.samplers.Sampler;
-import io.opentelemetry.semconv.resource.attributes.ResourceAttributes;
+import io.opentelemetry.semconv.ResourceAttributes;
import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_BATCH_SIZE_SETTING;
import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_DELAY_SETTING;
@@ -44,11 +46,11 @@ private OTelResourceProvider() {}
* Creates OpenTelemetry instance with default configuration
* @param telemetrySettings telemetry settings
* @param settings cluster settings
- * @return OpenTelemetry instance
+ * @return OpenTelemetrySdk instance
*/
- public static OpenTelemetry get(TelemetrySettings telemetrySettings, Settings settings) {
+ public static OpenTelemetrySdk get(TelemetrySettings telemetrySettings, Settings settings) {
return AccessController.doPrivileged(
- (PrivilegedAction) () -> get(
+ (PrivilegedAction) () -> get(
settings,
OTelSpanExporterFactory.create(settings),
ContextPropagators.create(W3CTraceContextPropagator.getInstance()),
@@ -63,17 +65,46 @@ public static OpenTelemetry get(TelemetrySettings telemetrySettings, Settings se
* @param spanExporter span exporter instance
* @param contextPropagators context propagator instance
* @param sampler sampler instance
- * @return Opentelemetry instance
+ * @return OpenTelemetrySdk instance
*/
- public static OpenTelemetry get(Settings settings, SpanExporter spanExporter, ContextPropagators contextPropagators, Sampler sampler) {
+ public static OpenTelemetrySdk get(
+ Settings settings,
+ SpanExporter spanExporter,
+ ContextPropagators contextPropagators,
+ Sampler sampler
+ ) {
Resource resource = Resource.create(Attributes.of(ResourceAttributes.SERVICE_NAME, "OpenSearch"));
- SdkTracerProvider sdkTracerProvider = SdkTracerProvider.builder()
+ SdkTracerProvider sdkTracerProvider = createSdkTracerProvider(settings, spanExporter, sampler, resource);
+ SdkMeterProvider sdkMeterProvider = createSdkMetricProvider(settings, resource);
+ return OpenTelemetrySdk.builder()
+ .setTracerProvider(sdkTracerProvider)
+ .setMeterProvider(sdkMeterProvider)
+ .setPropagators(contextPropagators)
+ .buildAndRegisterGlobal();
+ }
+
+ private static SdkMeterProvider createSdkMetricProvider(Settings settings, Resource resource) {
+ return SdkMeterProvider.builder()
+ .setResource(resource)
+ .registerMetricReader(
+ PeriodicMetricReader.builder(OTelMetricsExporterFactory.create(settings))
+ .setInterval(TelemetrySettings.METRICS_PUBLISH_INTERVAL_SETTING.get(settings).getSeconds(), TimeUnit.SECONDS)
+ .build()
+ )
+ .build();
+ }
+
+ private static SdkTracerProvider createSdkTracerProvider(
+ Settings settings,
+ SpanExporter spanExporter,
+ Sampler sampler,
+ Resource resource
+ ) {
+ return SdkTracerProvider.builder()
.addSpanProcessor(spanProcessor(settings, spanExporter))
.setResource(resource)
.setSampler(sampler)
.build();
-
- return OpenTelemetrySdk.builder().setTracerProvider(sdkTracerProvider).setPropagators(contextPropagators).buildAndRegisterGlobal();
}
private static BatchSpanProcessor spanProcessor(Settings settings, SpanExporter spanExporter) {
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java
index 8ad03d807d9da..fc917968579e1 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java
@@ -57,7 +57,9 @@ public void addAttribute(String key, Boolean value) {
@Override
public void setError(Exception exception) {
- delegateSpan.setStatus(StatusCode.ERROR, exception.getMessage());
+ if (exception != null) {
+ delegateSpan.setStatus(StatusCode.ERROR, exception.getMessage());
+ }
}
@Override
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java
index 282fabd43346b..0c697d2cc5e8c 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java
@@ -8,34 +8,39 @@
package org.opensearch.telemetry.tracing;
+import org.opensearch.common.concurrent.RefCountedReleasable;
import org.opensearch.telemetry.Telemetry;
import org.opensearch.telemetry.metrics.MetricsTelemetry;
+import org.opensearch.telemetry.metrics.OTelMetricsTelemetry;
+
+import io.opentelemetry.sdk.OpenTelemetrySdk;
/**
* Otel implementation of Telemetry
*/
public class OTelTelemetry implements Telemetry {
- private final TracingTelemetry tracingTelemetry;
- private final MetricsTelemetry metricsTelemetry;
+ private final RefCountedReleasable refCountedOpenTelemetry;
/**
* Creates Telemetry instance
- * @param tracingTelemetry tracing telemetry
- * @param metricsTelemetry metrics telemetry
+
+ */
+ /**
+ * Creates Telemetry instance
+ * @param refCountedOpenTelemetry open telemetry.
*/
- public OTelTelemetry(TracingTelemetry tracingTelemetry, MetricsTelemetry metricsTelemetry) {
- this.tracingTelemetry = tracingTelemetry;
- this.metricsTelemetry = metricsTelemetry;
+ public OTelTelemetry(RefCountedReleasable refCountedOpenTelemetry) {
+ this.refCountedOpenTelemetry = refCountedOpenTelemetry;
}
@Override
public TracingTelemetry getTracingTelemetry() {
- return tracingTelemetry;
+ return new OTelTracingTelemetry<>(refCountedOpenTelemetry, refCountedOpenTelemetry.get().getSdkTracerProvider());
}
@Override
public MetricsTelemetry getMetricsTelemetry() {
- return metricsTelemetry;
+ return new OTelMetricsTelemetry<>(refCountedOpenTelemetry, refCountedOpenTelemetry.get().getSdkMeterProvider());
}
}
diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java
index 53066ad4ad444..af39617a8c744 100644
--- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java
+++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java
@@ -8,41 +8,41 @@
package org.opensearch.telemetry.tracing;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import org.opensearch.common.concurrent.RefCountedReleasable;
+import org.opensearch.telemetry.OTelAttributesConverter;
+import org.opensearch.telemetry.OTelTelemetryPlugin;
import java.io.Closeable;
import java.io.IOException;
-import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.trace.TracerProvider;
import io.opentelemetry.context.Context;
+import io.opentelemetry.sdk.OpenTelemetrySdk;
/**
* OTel based Telemetry provider
*/
-public class OTelTracingTelemetry implements TracingTelemetry {
-
- private static final Logger logger = LogManager.getLogger(OTelTracingTelemetry.class);
- private final OpenTelemetry openTelemetry;
+public class OTelTracingTelemetry implements TracingTelemetry {
+ private final RefCountedReleasable refCountedOpenTelemetry;
+ private final T tracerProvider;
private final io.opentelemetry.api.trace.Tracer otelTracer;
/**
- * Creates OTel based Telemetry
- * @param openTelemetry OpenTelemetry instance
+ * Creates OTel based {@link TracingTelemetry}
+ * @param refCountedOpenTelemetry OpenTelemetry instance
+ * @param tracerProvider {@link TracerProvider} instance.
*/
- public OTelTracingTelemetry(OpenTelemetry openTelemetry) {
- this.openTelemetry = openTelemetry;
- this.otelTracer = openTelemetry.getTracer("os-tracer");
-
+ public OTelTracingTelemetry(RefCountedReleasable refCountedOpenTelemetry, T tracerProvider) {
+ this.refCountedOpenTelemetry = refCountedOpenTelemetry;
+ this.refCountedOpenTelemetry.incRef();
+ this.tracerProvider = tracerProvider;
+ this.otelTracer = tracerProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME);
}
@Override
- public void close() {
- try {
- ((Closeable) openTelemetry).close();
- } catch (IOException e) {
- logger.warn("Error while closing Opentelemetry", e);
- }
+ public void close() throws IOException {
+ tracerProvider.close();
+ refCountedOpenTelemetry.close();
}
@Override
@@ -52,7 +52,7 @@ public Span createSpan(SpanCreationContext spanCreationContext, Span parentSpan)
@Override
public TracingContextPropagator getContextPropagator() {
- return new OTelTracingContextPropagator(openTelemetry);
+ return new OTelTracingContextPropagator(refCountedOpenTelemetry.get());
}
private Span createOtelSpan(SpanCreationContext spanCreationContext, Span parentSpan) {
diff --git a/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy b/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy
index 726db3d3f4700..9d529ed5a2a56 100644
--- a/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy
+++ b/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy
@@ -11,6 +11,7 @@ grant {
permission java.lang.RuntimePermission "accessDeclaredMembers";
permission java.net.NetPermission "getProxySelector";
permission java.net.SocketPermission "*", "connect,resolve";
+ permission java.util.PropertyPermission "*", "read,write";
};
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java
index 8c2b5d14733e2..2fcf89947e537 100644
--- a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java
@@ -12,12 +12,15 @@
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.FeatureFlags;
+import org.opensearch.telemetry.metrics.MetricsTelemetry;
+import org.opensearch.telemetry.metrics.OTelMetricsTelemetry;
import org.opensearch.telemetry.tracing.OTelTracingTelemetry;
import org.opensearch.telemetry.tracing.TracingTelemetry;
import org.opensearch.test.OpenSearchTestCase;
import org.junit.After;
import org.junit.Before;
+import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
@@ -25,6 +28,7 @@
import java.util.Set;
import static org.opensearch.telemetry.OTelTelemetryPlugin.OTEL_TRACER_NAME;
+import static org.opensearch.telemetry.OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING;
import static org.opensearch.telemetry.OTelTelemetrySettings.OTEL_TRACER_SPAN_EXPORTER_CLASS_SETTING;
import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_BATCH_SIZE_SETTING;
import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_DELAY_SETTING;
@@ -34,41 +38,47 @@
public class OTelTelemetryPluginTests extends OpenSearchTestCase {
- private OTelTelemetryPlugin oTelTracerModulePlugin;
+ private OTelTelemetryPlugin oTelTelemetryPlugin;
private Optional telemetry;
private TracingTelemetry tracingTelemetry;
+ private MetricsTelemetry metricsTelemetry;
+
@Before
public void setup() {
// TRACER_EXPORTER_DELAY_SETTING should always be less than 10 seconds because
// io.opentelemetry.sdk.OpenTelemetrySdk.close waits only for 10 seconds for shutdown to complete.
Settings settings = Settings.builder().put(TRACER_EXPORTER_DELAY_SETTING.getKey(), "1s").build();
- oTelTracerModulePlugin = new OTelTelemetryPlugin(settings);
- telemetry = oTelTracerModulePlugin.getTelemetry(
+ oTelTelemetryPlugin = new OTelTelemetryPlugin(settings);
+ telemetry = oTelTelemetryPlugin.getTelemetry(
new TelemetrySettings(Settings.EMPTY, new ClusterSettings(settings, Set.of(TRACER_ENABLED_SETTING, TRACER_SAMPLER_PROBABILITY)))
);
tracingTelemetry = telemetry.get().getTracingTelemetry();
+ metricsTelemetry = telemetry.get().getMetricsTelemetry();
}
public void testGetTelemetry() {
Set> allTracerSettings = new HashSet<>();
ClusterSettings.FEATURE_FLAGGED_CLUSTER_SETTINGS.get(List.of(FeatureFlags.TELEMETRY)).stream().forEach((allTracerSettings::add));
- assertEquals(OTEL_TRACER_NAME, oTelTracerModulePlugin.getName());
+ assertEquals(OTEL_TRACER_NAME, oTelTelemetryPlugin.getName());
assertTrue(tracingTelemetry instanceof OTelTracingTelemetry);
+ assertTrue(metricsTelemetry instanceof OTelMetricsTelemetry);
assertEquals(
Arrays.asList(
TRACER_EXPORTER_BATCH_SIZE_SETTING,
TRACER_EXPORTER_DELAY_SETTING,
TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING,
- OTEL_TRACER_SPAN_EXPORTER_CLASS_SETTING
+ OTEL_TRACER_SPAN_EXPORTER_CLASS_SETTING,
+ OTEL_METRICS_EXPORTER_CLASS_SETTING
),
- oTelTracerModulePlugin.getSettings()
+ oTelTelemetryPlugin.getSettings()
);
}
@After
- public void cleanup() {
+ public void cleanup() throws IOException {
tracingTelemetry.close();
+ metricsTelemetry.close();
}
}
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetryTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetryTests.java
new file mode 100644
index 0000000000000..9de575b69774a
--- /dev/null
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/OTelMetricsTelemetryTests.java
@@ -0,0 +1,121 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics;
+
+import org.opensearch.common.concurrent.RefCountedReleasable;
+import org.opensearch.telemetry.OTelAttributesConverter;
+import org.opensearch.telemetry.OTelTelemetryPlugin;
+import org.opensearch.telemetry.metrics.tags.Tags;
+import org.opensearch.test.OpenSearchTestCase;
+
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.metrics.DoubleCounter;
+import io.opentelemetry.api.metrics.DoubleCounterBuilder;
+import io.opentelemetry.api.metrics.DoubleUpDownCounter;
+import io.opentelemetry.api.metrics.DoubleUpDownCounterBuilder;
+import io.opentelemetry.api.metrics.LongCounterBuilder;
+import io.opentelemetry.api.metrics.LongUpDownCounterBuilder;
+import io.opentelemetry.api.metrics.Meter;
+import io.opentelemetry.api.metrics.MeterProvider;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class OTelMetricsTelemetryTests extends OpenSearchTestCase {
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public void testCounter() {
+ String counterName = "test-counter";
+ String description = "test";
+ String unit = "1";
+ Meter mockMeter = mock(Meter.class);
+ OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
+ DoubleCounter mockOTelDoubleCounter = mock(DoubleCounter.class);
+ LongCounterBuilder mockOTelLongCounterBuilder = mock(LongCounterBuilder.class);
+ DoubleCounterBuilder mockOTelDoubleCounterBuilder = mock(DoubleCounterBuilder.class);
+ MeterProvider meterProvider = mock(MeterProvider.class);
+ when(meterProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockMeter);
+ MetricsTelemetry metricsTelemetry = new OTelMetricsTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ meterProvider
+ );
+ when(mockMeter.counterBuilder(counterName)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.setDescription(description)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.setUnit(unit)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.ofDoubles()).thenReturn(mockOTelDoubleCounterBuilder);
+ when(mockOTelDoubleCounterBuilder.build()).thenReturn(mockOTelDoubleCounter);
+
+ Counter counter = metricsTelemetry.createCounter(counterName, description, unit);
+ counter.add(1.0);
+ verify(mockOTelDoubleCounter).add(1.0);
+ Tags tags = Tags.create().addTag("test", "test");
+ counter.add(2.0, tags);
+ verify(mockOTelDoubleCounter).add(2.0, OTelAttributesConverter.convert(tags));
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public void testCounterNegativeValue() {
+ String counterName = "test-counter";
+ String description = "test";
+ String unit = "1";
+ OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
+ Meter mockMeter = mock(Meter.class);
+ DoubleCounter mockOTelDoubleCounter = mock(DoubleCounter.class);
+ LongCounterBuilder mockOTelLongCounterBuilder = mock(LongCounterBuilder.class);
+ DoubleCounterBuilder mockOTelDoubleCounterBuilder = mock(DoubleCounterBuilder.class);
+
+ MeterProvider meterProvider = mock(MeterProvider.class);
+ when(meterProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockMeter);
+ MetricsTelemetry metricsTelemetry = new OTelMetricsTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ meterProvider
+ );
+ when(mockMeter.counterBuilder(counterName)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.setDescription(description)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.setUnit(unit)).thenReturn(mockOTelLongCounterBuilder);
+ when(mockOTelLongCounterBuilder.ofDoubles()).thenReturn(mockOTelDoubleCounterBuilder);
+ when(mockOTelDoubleCounterBuilder.build()).thenReturn(mockOTelDoubleCounter);
+
+ Counter counter = metricsTelemetry.createCounter(counterName, description, unit);
+ counter.add(-1.0);
+ verify(mockOTelDoubleCounter).add(-1.0);
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public void testUpDownCounter() {
+ String counterName = "test-counter";
+ String description = "test";
+ String unit = "1";
+ OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
+ Meter mockMeter = mock(Meter.class);
+ DoubleUpDownCounter mockOTelUpDownDoubleCounter = mock(DoubleUpDownCounter.class);
+ LongUpDownCounterBuilder mockOTelLongUpDownCounterBuilder = mock(LongUpDownCounterBuilder.class);
+ DoubleUpDownCounterBuilder mockOTelDoubleUpDownCounterBuilder = mock(DoubleUpDownCounterBuilder.class);
+
+ MeterProvider meterProvider = mock(MeterProvider.class);
+ when(meterProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockMeter);
+ MetricsTelemetry metricsTelemetry = new OTelMetricsTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ meterProvider
+ );
+ when(mockMeter.upDownCounterBuilder(counterName)).thenReturn(mockOTelLongUpDownCounterBuilder);
+ when(mockOTelLongUpDownCounterBuilder.setDescription(description)).thenReturn(mockOTelLongUpDownCounterBuilder);
+ when(mockOTelLongUpDownCounterBuilder.setUnit(unit)).thenReturn(mockOTelLongUpDownCounterBuilder);
+ when(mockOTelLongUpDownCounterBuilder.ofDoubles()).thenReturn(mockOTelDoubleUpDownCounterBuilder);
+ when(mockOTelDoubleUpDownCounterBuilder.build()).thenReturn(mockOTelUpDownDoubleCounter);
+
+ Counter counter = metricsTelemetry.createUpDownCounter(counterName, description, unit);
+ counter.add(1.0);
+ verify(mockOTelUpDownDoubleCounter).add(1.0);
+ Tags tags = Tags.create().addTag("test", "test");
+ counter.add(-2.0, tags);
+ verify(mockOTelUpDownDoubleCounter).add((-2.0), OTelAttributesConverter.convert(tags));
+ }
+}
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/DummyMetricExporter.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/DummyMetricExporter.java
new file mode 100644
index 0000000000000..65c52911dbef9
--- /dev/null
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/DummyMetricExporter.java
@@ -0,0 +1,39 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics.exporter;
+
+import java.util.Collection;
+
+import io.opentelemetry.sdk.common.CompletableResultCode;
+import io.opentelemetry.sdk.metrics.InstrumentType;
+import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
+import io.opentelemetry.sdk.metrics.data.MetricData;
+import io.opentelemetry.sdk.metrics.export.MetricExporter;
+
+public class DummyMetricExporter implements MetricExporter {
+ @Override
+ public CompletableResultCode export(Collection metrics) {
+ return null;
+ }
+
+ @Override
+ public CompletableResultCode flush() {
+ return null;
+ }
+
+ @Override
+ public CompletableResultCode shutdown() {
+ return null;
+ }
+
+ @Override
+ public AggregationTemporality getAggregationTemporality(InstrumentType instrumentType) {
+ return null;
+ }
+}
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactoryTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactoryTests.java
new file mode 100644
index 0000000000000..e68da030bfb52
--- /dev/null
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/metrics/exporter/OTelMetricsExporterFactoryTests.java
@@ -0,0 +1,78 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.telemetry.metrics.exporter;
+
+import org.opensearch.common.settings.Settings;
+import org.opensearch.telemetry.OTelTelemetrySettings;
+import org.opensearch.test.OpenSearchTestCase;
+
+import io.opentelemetry.exporter.logging.LoggingMetricExporter;
+import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter;
+import io.opentelemetry.sdk.metrics.export.MetricExporter;
+
+public class OTelMetricsExporterFactoryTests extends OpenSearchTestCase {
+
+ public void testMetricsExporterDefault() {
+ Settings settings = Settings.builder().build();
+ MetricExporter metricExporter = OTelMetricsExporterFactory.create(settings);
+ assertTrue(metricExporter instanceof LoggingMetricExporter);
+ }
+
+ public void testMetricsExporterLogging() {
+ Settings settings = Settings.builder()
+ .put(
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(),
+ "io.opentelemetry.exporter.logging.LoggingMetricExporter"
+ )
+ .build();
+ MetricExporter metricExporter = OTelMetricsExporterFactory.create(settings);
+ assertTrue(metricExporter instanceof LoggingMetricExporter);
+ }
+
+ public void testMetricExporterInvalid() {
+ Settings settings = Settings.builder().put(OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(), "abc").build();
+ assertThrows(IllegalArgumentException.class, () -> OTelMetricsExporterFactory.create(settings));
+ }
+
+ public void testMetricExporterNoCreateFactoryMethod() {
+ Settings settings = Settings.builder()
+ .put(
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(),
+ "org.opensearch.telemetry.metrics.exporter.DummyMetricExporter"
+ )
+ .build();
+ IllegalStateException exception = assertThrows(IllegalStateException.class, () -> OTelMetricsExporterFactory.create(settings));
+ assertEquals(
+ "MetricExporter instantiation failed for class [org.opensearch.telemetry.metrics.exporter.DummyMetricExporter]",
+ exception.getMessage()
+ );
+ }
+
+ public void testMetricExporterNonMetricExporterClass() {
+ Settings settings = Settings.builder()
+ .put(OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(), "java.lang.String")
+ .build();
+ IllegalStateException exception = assertThrows(IllegalStateException.class, () -> OTelMetricsExporterFactory.create(settings));
+ assertEquals("MetricExporter instantiation failed for class [java.lang.String]", exception.getMessage());
+ assertTrue(exception.getCause() instanceof NoSuchMethodError);
+
+ }
+
+ public void testMetricExporterGetDefaultMethod() {
+ Settings settings = Settings.builder()
+ .put(
+ OTelTelemetrySettings.OTEL_METRICS_EXPORTER_CLASS_SETTING.getKey(),
+ "io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter"
+ )
+ .build();
+
+ assertTrue(OTelMetricsExporterFactory.create(settings) instanceof OtlpGrpcMetricExporter);
+ }
+
+}
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelAttributesConverterTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelAttributesConverterTests.java
index d992daec1b7bb..ee67384d01759 100644
--- a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelAttributesConverterTests.java
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelAttributesConverterTests.java
@@ -8,6 +8,8 @@
package org.opensearch.telemetry.tracing;
+import org.opensearch.telemetry.OTelAttributesConverter;
+import org.opensearch.telemetry.metrics.tags.Tags;
import org.opensearch.telemetry.tracing.attributes.Attributes;
import org.opensearch.test.OpenSearchTestCase;
@@ -19,13 +21,13 @@
public class OTelAttributesConverterTests extends OpenSearchTestCase {
public void testConverterNullAttributes() {
- io.opentelemetry.api.common.Attributes otelAttributes = OTelAttributesConverter.convert(null);
+ io.opentelemetry.api.common.Attributes otelAttributes = OTelAttributesConverter.convert((Attributes) null);
assertEquals(0, otelAttributes.size());
}
public void testConverterEmptyAttributes() {
Attributes attributes = Attributes.EMPTY;
- io.opentelemetry.api.common.Attributes otelAttributes = OTelAttributesConverter.convert(null);
+ io.opentelemetry.api.common.Attributes otelAttributes = OTelAttributesConverter.convert(attributes);
assertEquals(0, otelAttributes.size());
}
@@ -47,4 +49,12 @@ public void testConverterMultipleAttributes() {
assertEquals(4, otelAttributes.size());
otelAttributes.asMap().forEach((x, y) -> assertEquals(attributeMap.get(x.getKey()), y));
}
+
+ public void testConverterMultipleTags() {
+ Tags tags = Tags.create().addTag("key1", 1l).addTag("key2", 1.0).addTag("key3", true).addTag("key4", "value4");
+ Map tagsMap = tags.getTagsMap();
+ io.opentelemetry.api.common.Attributes otelAttributes = OTelAttributesConverter.convert(tags);
+ assertEquals(4, otelAttributes.size());
+ otelAttributes.asMap().forEach((x, y) -> assertEquals(tagsMap.get(x.getKey()), y));
+ }
}
diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java
index 505756318ff62..1f0c2f674e655 100644
--- a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java
+++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java
@@ -8,16 +8,16 @@
package org.opensearch.telemetry.tracing;
+import org.opensearch.common.concurrent.RefCountedReleasable;
+import org.opensearch.telemetry.OTelTelemetryPlugin;
import org.opensearch.telemetry.tracing.attributes.Attributes;
import org.opensearch.test.OpenSearchTestCase;
-import java.util.Collections;
-import java.util.Map;
-
import io.opentelemetry.api.OpenTelemetry;
import io.opentelemetry.api.common.AttributesBuilder;
import io.opentelemetry.api.trace.SpanBuilder;
import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.api.trace.TracerProvider;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
@@ -26,29 +26,34 @@
import static org.mockito.Mockito.when;
public class OTelTracingTelemetryTests extends OpenSearchTestCase {
-
+ @SuppressWarnings({ "rawtypes", "unchecked" })
public void testCreateSpanWithoutParent() {
OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
Tracer mockTracer = mock(Tracer.class);
- when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer);
+ TracerProvider mockTracerProvider = mock(TracerProvider.class);
+ when(mockTracerProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockTracer);
SpanBuilder mockSpanBuilder = mock(SpanBuilder.class);
when(mockTracer.spanBuilder("span_name")).thenReturn(mockSpanBuilder);
when(mockSpanBuilder.setAllAttributes(any(io.opentelemetry.api.common.Attributes.class))).thenReturn(mockSpanBuilder);
when(mockSpanBuilder.startSpan()).thenReturn(mock(io.opentelemetry.api.trace.Span.class));
when(mockSpanBuilder.setSpanKind(any(io.opentelemetry.api.trace.SpanKind.class))).thenReturn(mockSpanBuilder);
- Map attributeMap = Collections.singletonMap("name", "value");
Attributes attributes = Attributes.create().addAttribute("name", "value");
- TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry);
+ TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ mockTracerProvider
+ );
Span span = tracingTelemetry.createSpan(SpanCreationContext.internal().name("span_name").attributes(attributes), null);
verify(mockSpanBuilder, never()).setParent(any());
verify(mockSpanBuilder).setAllAttributes(createAttribute(attributes));
assertNull(span.getParentSpan());
}
+ @SuppressWarnings({ "rawtypes", "unchecked" })
public void testCreateSpanWithParent() {
OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
Tracer mockTracer = mock(Tracer.class);
- when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer);
+ TracerProvider mockTracerProvider = mock(TracerProvider.class);
+ when(mockTracerProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockTracer);
SpanBuilder mockSpanBuilder = mock(SpanBuilder.class);
when(mockTracer.spanBuilder("span_name")).thenReturn(mockSpanBuilder);
when(mockSpanBuilder.setParent(any())).thenReturn(mockSpanBuilder);
@@ -58,7 +63,10 @@ public void testCreateSpanWithParent() {
Span parentSpan = new OTelSpan("parent_span", mock(io.opentelemetry.api.trace.Span.class), null);
- TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry);
+ TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ mockTracerProvider
+ );
Attributes attributes = Attributes.create().addAttribute("name", 1l);
Span span = tracingTelemetry.createSpan(SpanCreationContext.internal().name("span_name").attributes(attributes), parentSpan);
@@ -69,10 +77,12 @@ public void testCreateSpanWithParent() {
assertEquals("parent_span", span.getParentSpan().getSpanName());
}
+ @SuppressWarnings({ "rawtypes", "unchecked" })
public void testCreateSpanWithParentWithMultipleAttributes() {
OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
Tracer mockTracer = mock(Tracer.class);
- when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer);
+ TracerProvider mockTracerProvider = mock(TracerProvider.class);
+ when(mockTracerProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockTracer);
SpanBuilder mockSpanBuilder = mock(SpanBuilder.class);
when(mockTracer.spanBuilder("span_name")).thenReturn(mockSpanBuilder);
when(mockSpanBuilder.setParent(any())).thenReturn(mockSpanBuilder);
@@ -82,7 +92,10 @@ public void testCreateSpanWithParentWithMultipleAttributes() {
Span parentSpan = new OTelSpan("parent_span", mock(io.opentelemetry.api.trace.Span.class), null);
- TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry);
+ TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ mockTracerProvider
+ );
Attributes attributes = Attributes.create()
.addAttribute("key1", 1l)
.addAttribute("key2", 2.0)
@@ -115,12 +128,17 @@ private io.opentelemetry.api.common.Attributes createAttributeLong(Attributes at
return attributesBuilder.build();
}
+ @SuppressWarnings({ "rawtypes", "unchecked" })
public void testGetContextPropagator() {
OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class);
Tracer mockTracer = mock(Tracer.class);
- when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer);
+ TracerProvider mockTracerProvider = mock(TracerProvider.class);
+ when(mockTracerProvider.get(OTelTelemetryPlugin.INSTRUMENTATION_SCOPE_NAME)).thenReturn(mockTracer);
- TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry);
+ TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(
+ new RefCountedReleasable("telemetry", mockOpenTelemetry, () -> {}),
+ mockTracerProvider
+ );
assertTrue(tracingTelemetry.getContextPropagator() instanceof OTelTracingContextPropagator);
}
diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..aaf2e35302d77
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-buffer-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+39b05d2d4027971bf99111a9be1d7035a116bb55
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.97.Final.jar.sha1
deleted file mode 100644
index 8430355365996..0000000000000
--- a/plugins/transport-nio/licenses/netty-buffer-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-f8f3d8644afa5e6e1a40a3a6aeb9d9aa970ecb4f
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..a77333ea8ae47
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-codec-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+9c3c71e7cf3b8ce3bfc9fa52a524b9ca7ddf259c
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.97.Final.jar.sha1
deleted file mode 100644
index 7a36dc1f2724f..0000000000000
--- a/plugins/transport-nio/licenses/netty-codec-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-384ba4d75670befbedb45c4d3b497a93639c206d
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..6f26bf4e6a9b5
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-codec-http-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+992623e7d8f2d96e41faf1687bb963f5433e3517
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.97.Final.jar.sha1
deleted file mode 100644
index 37b78a32f741f..0000000000000
--- a/plugins/transport-nio/licenses/netty-codec-http-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-af78acec783ffd77c63d8aeecc21041fd39ac54f
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-common-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..d2ff72db60d1f
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-common-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+847f942381145de23f21c836d05b0677474271d3
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-common-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.97.Final.jar.sha1
deleted file mode 100644
index 1bdfec3aae6ba..0000000000000
--- a/plugins/transport-nio/licenses/netty-common-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-7cceacaf11df8dc63f23d0fb58e9d4640fc88404
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..f12a6046e96d0
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-handler-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+4c0acdb8bb73647ebb3847ac2d503d53d72c02b4
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.97.Final.jar.sha1
deleted file mode 100644
index 8b7b50a6fc9c6..0000000000000
--- a/plugins/transport-nio/licenses/netty-handler-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-abb86c6906bf512bf2b797a41cd7d2e8d3cd7c36
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..8e4179ba15942
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-resolver-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+fe62f9ccd41b8660d07639dbbab8ae1edd6f2720
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.97.Final.jar.sha1
deleted file mode 100644
index 032959e98d009..0000000000000
--- a/plugins/transport-nio/licenses/netty-resolver-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-cec8348108dc76c47cf87c669d514be52c922144
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.100.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..ab2819da570fd
--- /dev/null
+++ b/plugins/transport-nio/licenses/netty-transport-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+6620fbfb47667a5eb6050e35c7b4c88000bcd77f
\ No newline at end of file
diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.97.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.97.Final.jar.sha1
deleted file mode 100644
index 107863c1b3c9d..0000000000000
--- a/plugins/transport-nio/licenses/netty-transport-4.1.97.Final.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-f37380d23c9bb079bc702910833b2fd532c9abd0
\ No newline at end of file
diff --git a/plugins/transport-nio/src/main/java/org/opensearch/http/nio/NioHttpRequest.java b/plugins/transport-nio/src/main/java/org/opensearch/http/nio/NioHttpRequest.java
index d25ef33c2ce29..5abd6f2710198 100644
--- a/plugins/transport-nio/src/main/java/org/opensearch/http/nio/NioHttpRequest.java
+++ b/plugins/transport-nio/src/main/java/org/opensearch/http/nio/NioHttpRequest.java
@@ -257,7 +257,7 @@ public FullHttpRequest nettyRequest() {
/**
* A wrapper of {@link HttpHeaders} that implements a map to prevent copying unnecessarily. This class does not support modifications
* and due to the underlying implementation, it performs case insensitive lookups of key to values.
- *
+ *
* It is important to note that this implementation does have some downsides in that each invocation of the
* {@link #values()} and {@link #entrySet()} methods will perform a copy of the values in the HttpHeaders rather than returning a
* view of the underlying values.
diff --git a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransport.java b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransport.java
index dfa72d6d59a0d..55920bab4efd3 100644
--- a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransport.java
+++ b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransport.java
@@ -52,6 +52,7 @@
import org.opensearch.nio.NioSelector;
import org.opensearch.nio.NioSocketChannel;
import org.opensearch.nio.ServerChannelContext;
+import org.opensearch.telemetry.tracing.Tracer;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.TcpTransport;
import org.opensearch.transport.TransportSettings;
@@ -84,9 +85,10 @@ protected NioTransport(
PageCacheRecycler pageCacheRecycler,
NamedWriteableRegistry namedWriteableRegistry,
CircuitBreakerService circuitBreakerService,
- NioGroupFactory groupFactory
+ NioGroupFactory groupFactory,
+ Tracer tracer
) {
- super(settings, version, threadPool, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, networkService);
+ super(settings, version, threadPool, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, networkService, tracer);
this.pageAllocator = new PageAllocator(pageCacheRecycler);
this.groupFactory = groupFactory;
}
diff --git a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransportPlugin.java b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransportPlugin.java
index ec266d76eff3d..d4be876867651 100644
--- a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransportPlugin.java
+++ b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/NioTransportPlugin.java
@@ -91,7 +91,8 @@ public Map> getTransports(
PageCacheRecycler pageCacheRecycler,
CircuitBreakerService circuitBreakerService,
NamedWriteableRegistry namedWriteableRegistry,
- NetworkService networkService
+ NetworkService networkService,
+ Tracer tracer
) {
return Collections.singletonMap(
NIO_TRANSPORT_NAME,
@@ -103,7 +104,8 @@ public Map> getTransports(
pageCacheRecycler,
namedWriteableRegistry,
circuitBreakerService,
- getNioGroupFactory(settings)
+ getNioGroupFactory(settings),
+ tracer
)
);
}
diff --git a/plugins/transport-nio/src/test/java/org/opensearch/transport/nio/SimpleNioTransportTests.java b/plugins/transport-nio/src/test/java/org/opensearch/transport/nio/SimpleNioTransportTests.java
index 24cc38c17a9d1..f5d1c618f5ace 100644
--- a/plugins/transport-nio/src/test/java/org/opensearch/transport/nio/SimpleNioTransportTests.java
+++ b/plugins/transport-nio/src/test/java/org/opensearch/transport/nio/SimpleNioTransportTests.java
@@ -44,6 +44,7 @@
import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
import org.opensearch.core.common.transport.TransportAddress;
import org.opensearch.core.indices.breaker.NoneCircuitBreakerService;
+import org.opensearch.telemetry.tracing.noop.NoopTracer;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.test.transport.StubbableTransport;
import org.opensearch.transport.AbstractSimpleTransportTestCase;
@@ -81,7 +82,8 @@ protected Transport build(Settings settings, final Version version, ClusterSetti
new MockPageCacheRecycler(settings),
namedWriteableRegistry,
new NoneCircuitBreakerService(),
- new NioGroupFactory(settings, logger)
+ new NioGroupFactory(settings, logger),
+ NoopTracer.INSTANCE
) {
@Override
diff --git a/plugins/transport-reactor-netty4/build.gradle b/plugins/transport-reactor-netty4/build.gradle
new file mode 100644
index 0000000000000..7d7eb330b4a55
--- /dev/null
+++ b/plugins/transport-reactor-netty4/build.gradle
@@ -0,0 +1,264 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ *
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+import org.opensearch.gradle.info.BuildParams
+import org.opensearch.gradle.test.RestIntegTestTask
+import org.opensearch.gradle.test.TestTask
+import org.opensearch.gradle.test.rest.JavaRestTestPlugin
+import org.opensearch.gradle.test.InternalClusterTestPlugin
+
+apply plugin: 'opensearch.yaml-rest-test'
+apply plugin: 'opensearch.java-rest-test'
+apply plugin: 'opensearch.internal-cluster-test'
+
+// The transport-reactor-netty4 plugin is published to maven
+apply plugin: 'opensearch.publish'
+
+opensearchplugin {
+ description 'Reactor Netty 4 based transport implementation'
+ classname 'org.opensearch.transport.reactor.ReactorNetty4Plugin'
+ hasClientJar = true
+}
+
+dependencies {
+ // network stack
+ api "io.netty:netty-buffer:${versions.netty}"
+ api "io.netty:netty-codec:${versions.netty}"
+ api "io.netty:netty-codec-dns:${versions.netty}"
+ api "io.netty:netty-codec-http:${versions.netty}"
+ api "io.netty:netty-codec-http2:${versions.netty}"
+ api "io.netty:netty-common:${versions.netty}"
+ api "io.netty:netty-handler:${versions.netty}"
+ api "io.netty:netty-resolver-dns:${versions.netty}"
+ api "io.netty:netty-resolver:${versions.netty}"
+ api "io.netty:netty-transport:${versions.netty}"
+ api "io.netty:netty-transport-native-unix-common:${versions.netty}"
+
+ api "io.projectreactor.netty:reactor-netty-http:${versions.reactor_netty}"
+ api "io.projectreactor.netty:reactor-netty-core:${versions.reactor_netty}"
+
+ testImplementation "org.apache.logging.log4j:log4j-slf4j-impl:${versions.log4j}"
+ testImplementation "io.projectreactor:reactor-test:${versions.reactor}"
+ testImplementation project(":modules:transport-netty4")
+}
+
+restResources {
+ restApi {
+ includeCore '_common', 'cluster', 'nodes'
+ }
+}
+
+tasks.named("dependencyLicenses").configure {
+ mapping from: /netty-.*/, to: 'netty'
+ mapping from: /reactor-.*/, to: 'reactor'
+}
+
+// TODO: Remove that once we have a complete test suite
+testingConventions.enabled = false
+
+test {
+ /*
+ * We have to disable setting the number of available processors as tests in the same JVM randomize processors and will step on each
+ * other if we allow them to set the number of available processors as it's set-once in Netty.
+ */
+ systemProperty 'opensearch.set.netty.runtime.available.processors', 'false'
+}
+
+internalClusterTest {
+ systemProperty 'opensearch.set.netty.runtime.available.processors', 'false'
+}
+
+javaRestTest {
+ systemProperty 'opensearch.set.netty.runtime.available.processors', 'false'
+}
+
+thirdPartyAudit {
+ ignoreMissingClasses(
+ 'com.aayushatharva.brotli4j.Brotli4jLoader',
+ 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Status',
+ 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Wrapper',
+ 'com.aayushatharva.brotli4j.encoder.BrotliEncoderChannel',
+ 'com.aayushatharva.brotli4j.encoder.Encoder$Mode',
+ 'com.aayushatharva.brotli4j.encoder.Encoder$Parameters',
+ // classes are missing
+
+ // from io.netty.logging.CommonsLoggerFactory (netty)
+ 'org.apache.commons.logging.Log',
+ 'org.apache.commons.logging.LogFactory',
+
+ // from Log4j (deliberate, Netty will fallback to Log4j 2)
+ 'org.apache.log4j.Level',
+ 'org.apache.log4j.Logger',
+
+ // from io.netty.handler.ssl.OpenSslEngine (netty)
+ 'io.netty.internal.tcnative.Buffer',
+ 'io.netty.internal.tcnative.CertificateCompressionAlgo',
+ 'io.netty.internal.tcnative.Library',
+ 'io.netty.internal.tcnative.SSL',
+ 'io.netty.internal.tcnative.SSLContext',
+ 'io.netty.internal.tcnative.SSLPrivateKeyMethod',
+
+ // from io.netty.handler.ssl.util.BouncyCastleSelfSignedCertGenerator (netty)
+ 'org.bouncycastle.cert.X509v3CertificateBuilder',
+ 'org.bouncycastle.cert.jcajce.JcaX509CertificateConverter',
+ 'org.bouncycastle.operator.jcajce.JcaContentSignerBuilder',
+ 'org.bouncycastle.openssl.PEMEncryptedKeyPair',
+ 'org.bouncycastle.openssl.PEMParser',
+ 'org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter',
+ 'org.bouncycastle.openssl.jcajce.JceOpenSSLPKCS8DecryptorProviderBuilder',
+ 'org.bouncycastle.openssl.jcajce.JcePEMDecryptorProviderBuilder',
+ 'org.bouncycastle.pkcs.PKCS8EncryptedPrivateKeyInfo',
+
+ // from io.netty.handler.ssl.JettyNpnSslEngine (netty)
+ 'org.eclipse.jetty.npn.NextProtoNego$ClientProvider',
+ 'org.eclipse.jetty.npn.NextProtoNego$ServerProvider',
+ 'org.eclipse.jetty.npn.NextProtoNego',
+
+ // from io.netty.handler.codec.marshalling.ChannelBufferByteInput (netty)
+ 'org.jboss.marshalling.ByteInput',
+
+ // from io.netty.handler.codec.marshalling.ChannelBufferByteOutput (netty)
+ 'org.jboss.marshalling.ByteOutput',
+
+ // from io.netty.handler.codec.marshalling.CompatibleMarshallingEncoder (netty)
+ 'org.jboss.marshalling.Marshaller',
+
+ // from io.netty.handler.codec.marshalling.ContextBoundUnmarshallerProvider (netty)
+ 'org.jboss.marshalling.MarshallerFactory',
+ 'org.jboss.marshalling.MarshallingConfiguration',
+ 'org.jboss.marshalling.Unmarshaller',
+
+ // from io.netty.util.internal.logging.InternalLoggerFactory (netty) - it's optional
+ 'org.slf4j.helpers.FormattingTuple',
+ 'org.slf4j.helpers.MessageFormatter',
+ 'org.slf4j.Logger',
+ 'org.slf4j.LoggerFactory',
+ 'org.slf4j.spi.LocationAwareLogger',
+
+ 'com.google.protobuf.nano.CodedOutputByteBufferNano',
+ 'com.google.protobuf.nano.MessageNano',
+ 'com.ning.compress.BufferRecycler',
+ 'com.ning.compress.lzf.ChunkDecoder',
+ 'com.ning.compress.lzf.ChunkEncoder',
+ 'com.ning.compress.lzf.LZFChunk',
+ 'com.ning.compress.lzf.LZFEncoder',
+ 'com.ning.compress.lzf.util.ChunkDecoderFactory',
+ 'com.ning.compress.lzf.util.ChunkEncoderFactory',
+ 'lzma.sdk.lzma.Encoder',
+ 'net.jpountz.lz4.LZ4Compressor',
+ 'net.jpountz.lz4.LZ4Factory',
+ 'net.jpountz.lz4.LZ4FastDecompressor',
+ 'net.jpountz.xxhash.XXHash32',
+ 'net.jpountz.xxhash.XXHashFactory',
+ 'io.netty.internal.tcnative.AsyncSSLPrivateKeyMethod',
+ 'io.netty.internal.tcnative.AsyncTask',
+ 'io.netty.internal.tcnative.CertificateCallback',
+ 'io.netty.internal.tcnative.CertificateVerifier',
+ 'io.netty.internal.tcnative.ResultCallback',
+ 'io.netty.internal.tcnative.SessionTicketKey',
+ 'io.netty.internal.tcnative.SniHostNameMatcher',
+ 'io.netty.internal.tcnative.SSL',
+ 'io.netty.internal.tcnative.SSLSession',
+ 'io.netty.internal.tcnative.SSLSessionCache',
+ 'io.netty.channel.epoll.Epoll',
+ 'io.netty.channel.epoll.EpollDatagramChannel',
+ 'io.netty.channel.epoll.EpollServerSocketChannel',
+ 'io.netty.channel.epoll.EpollSocketChannel',
+ 'io.netty.channel.kqueue.KQueue',
+ 'io.netty.channel.kqueue.KQueueDatagramChannel',
+ 'io.netty.channel.kqueue.KQueueServerSocketChannel',
+ 'io.netty.channel.kqueue.KQueueSocketChannel',
+ 'io.netty.handler.codec.haproxy.HAProxyMessage',
+ 'io.netty.handler.codec.haproxy.HAProxyMessageDecoder',
+ 'io.netty.handler.proxy.ProxyHandler',
+ 'io.netty.incubator.channel.uring.IOUring',
+ 'io.netty.incubator.channel.uring.IOUringDatagramChannel',
+ 'io.netty.incubator.channel.uring.IOUringServerSocketChannel',
+ 'io.netty.incubator.channel.uring.IOUringSocketChannel',
+
+ 'org.eclipse.jetty.alpn.ALPN$ClientProvider',
+ 'org.eclipse.jetty.alpn.ALPN$ServerProvider',
+ 'org.eclipse.jetty.alpn.ALPN',
+
+ 'org.conscrypt.AllocatedBuffer',
+ 'org.conscrypt.BufferAllocator',
+ 'org.conscrypt.Conscrypt',
+ 'org.conscrypt.HandshakeListener',
+
+ 'reactor.blockhound.BlockHound$Builder',
+ 'reactor.blockhound.integration.BlockHoundIntegration',
+
+ 'io.micrometer.common.KeyValue',
+ 'io.micrometer.common.KeyValues',
+ 'io.micrometer.common.docs.KeyName',
+ 'io.micrometer.context.ContextAccessor',
+ 'io.micrometer.core.instrument.Counter',
+ 'io.micrometer.core.instrument.Counter$Builder',
+ 'io.micrometer.core.instrument.DistributionSummary',
+ 'io.micrometer.core.instrument.DistributionSummary$Builder',
+ 'io.micrometer.core.instrument.Gauge',
+ 'io.micrometer.core.instrument.Gauge$Builder',
+ 'io.micrometer.core.instrument.Meter',
+ 'io.micrometer.core.instrument.Meter$Type',
+ 'io.micrometer.core.instrument.MeterRegistry',
+ 'io.micrometer.core.instrument.Metrics',
+ 'io.micrometer.core.instrument.Tags',
+ 'io.micrometer.core.instrument.Timer',
+ 'io.micrometer.core.instrument.Timer$Builder',
+ 'io.micrometer.core.instrument.Timer$Sample',
+ 'io.micrometer.core.instrument.composite.CompositeMeterRegistry',
+ 'io.micrometer.core.instrument.docs.MeterDocumentation',
+ 'io.micrometer.core.instrument.observation.MeterObservationHandler',
+ 'io.micrometer.observation.Observation',
+ 'io.micrometer.observation.Observation$Context',
+ 'io.micrometer.observation.ObservationHandler',
+ 'io.micrometer.observation.ObservationRegistry',
+ 'io.micrometer.observation.ObservationRegistry$ObservationConfig',
+ 'io.micrometer.observation.docs.ObservationDocumentation',
+ 'io.micrometer.observation.transport.ReceiverContext',
+ 'io.micrometer.observation.transport.RequestReplyReceiverContext',
+ 'io.micrometer.observation.transport.RequestReplySenderContext',
+ 'io.micrometer.observation.transport.SenderContext',
+ 'io.micrometer.tracing.Span',
+ 'io.micrometer.tracing.Tracer',
+ 'io.micrometer.tracing.docs.SpanDocumentation',
+ 'io.micrometer.tracing.handler.DefaultTracingObservationHandler',
+ 'io.micrometer.tracing.handler.PropagatingReceiverTracingObservationHandler',
+ 'io.micrometer.tracing.handler.PropagatingSenderTracingObservationHandler',
+ 'io.micrometer.tracing.propagation.Propagator'
+ )
+
+ ignoreViolations(
+ 'io.netty.util.internal.PlatformDependent0',
+ 'io.netty.util.internal.PlatformDependent0$1',
+ 'io.netty.util.internal.PlatformDependent0$2',
+ 'io.netty.util.internal.PlatformDependent0$3',
+ 'io.netty.util.internal.PlatformDependent0$4',
+ 'io.netty.util.internal.PlatformDependent0$6',
+ 'io.netty.util.internal.shaded.org.jctools.queues.BaseLinkedQueueConsumerNodeRef',
+ 'io.netty.util.internal.shaded.org.jctools.queues.BaseLinkedQueueProducerNodeRef',
+ 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueColdProducerFields',
+ 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields',
+ 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields',
+ 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode',
+ 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField',
+ 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField',
+ 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField',
+ 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess',
+ 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$1',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$2',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$3',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$4',
+ 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$5'
+ )
+}
diff --git a/plugins/transport-reactor-netty4/licenses/netty-LICENSE.txt b/plugins/transport-reactor-netty4/licenses/netty-LICENSE.txt
new file mode 100644
index 0000000000000..d645695673349
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/plugins/transport-reactor-netty4/licenses/netty-NOTICE.txt b/plugins/transport-reactor-netty4/licenses/netty-NOTICE.txt
new file mode 100644
index 0000000000000..5bbf91a14de23
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-NOTICE.txt
@@ -0,0 +1,116 @@
+
+ The Netty Project
+ =================
+
+Please visit the Netty web site for more information:
+
+ * http://netty.io/
+
+Copyright 2011 The Netty Project
+
+The Netty Project licenses this file to you under the Apache License,
+version 2.0 (the "License"); you may not use this file except in compliance
+with the License. You may obtain a copy of the License at:
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+License for the specific language governing permissions and limitations
+under the License.
+
+Also, please refer to each LICENSE..txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+ * LICENSE:
+ * license/LICENSE.jsr166y.txt (Public Domain)
+ * HOMEPAGE:
+ * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.base64.txt (Public Domain)
+ * HOMEPAGE:
+ * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified version of 'JZlib', a re-implementation of
+zlib in pure Java, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jzlib.txt (BSD Style License)
+ * HOMEPAGE:
+ * http://www.jcraft.com/jzlib/
+
+This product contains a modified version of 'Webbit', a Java event based
+WebSocket and HTTP server:
+
+ * LICENSE:
+ * license/LICENSE.webbit.txt (BSD License)
+ * HOMEPAGE:
+ * https://github.com/joewalnes/webbit
+
+This product optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.protobuf.txt (New BSD License)
+ * HOMEPAGE:
+ * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.bouncycastle.txt (MIT License)
+ * HOMEPAGE:
+ * http://www.bouncycastle.org/
+
+This product optionally depends on 'SLF4J', a simple logging facade for Java,
+which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.slf4j.txt (MIT License)
+ * HOMEPAGE:
+ * http://www.slf4j.org/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.commons-logging.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework,
+which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.log4j.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://logging.apache.org/log4j/
+
+This product optionally depends on 'JBoss Logging', a logging framework,
+which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1)
+ * HOMEPAGE:
+ * http://anonsvn.jboss.org/repos/common/common-logging-spi/
+
+This product optionally depends on 'Apache Felix', an open source OSGi
+framework implementation, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.felix.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://felix.apache.org/
diff --git a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..aaf2e35302d77
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+39b05d2d4027971bf99111a9be1d7035a116bb55
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..a77333ea8ae47
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+9c3c71e7cf3b8ce3bfc9fa52a524b9ca7ddf259c
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..dfa4a0fbea94c
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+663b1b7bf3ff0f12fde4df20c72d9e94584ebffa
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..6f26bf4e6a9b5
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+992623e7d8f2d96e41faf1687bb963f5433e3517
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..bf5605151406e
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+cbf1a430ea44dbdedbcde16b185cbb95f28d72c7
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..d2ff72db60d1f
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+847f942381145de23f21c836d05b0677474271d3
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..f12a6046e96d0
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+4c0acdb8bb73647ebb3847ac2d503d53d72c02b4
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..8e4179ba15942
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+fe62f9ccd41b8660d07639dbbab8ae1edd6f2720
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..0948daa05fff6
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+62dbdf5f25eda75ea8456be1ed72b3fcb0d18774
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..ab2819da570fd
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+6620fbfb47667a5eb6050e35c7b4c88000bcd77f
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.100.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.100.Final.jar.sha1
new file mode 100644
index 0000000000000..30d7758302e37
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.100.Final.jar.sha1
@@ -0,0 +1 @@
+648ff5571022dbfa6789122e3872477bbf67fa7b
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/reactor-LICENSE.txt b/plugins/transport-reactor-netty4/licenses/reactor-LICENSE.txt
new file mode 100644
index 0000000000000..e5583c184e67a
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/reactor-LICENSE.txt
@@ -0,0 +1,201 @@
+Apache License
+ Version 2.0, January 2004
+ https://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "{}"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright {yyyy} {name of copyright owner}
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ https://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/plugins/discovery-ec2/licenses/reactive-streams-NOTICE.txt b/plugins/transport-reactor-netty4/licenses/reactor-NOTICE.txt
similarity index 100%
rename from plugins/discovery-ec2/licenses/reactive-streams-NOTICE.txt
rename to plugins/transport-reactor-netty4/licenses/reactor-NOTICE.txt
diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.12.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.12.jar.sha1
new file mode 100644
index 0000000000000..352d69396d0c9
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.12.jar.sha1
@@ -0,0 +1 @@
+378dc5a375e6440099e837b22cf4b01341cbe4ea
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.12.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.12.jar.sha1
new file mode 100644
index 0000000000000..1bcb0e0c52950
--- /dev/null
+++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.12.jar.sha1
@@ -0,0 +1 @@
+e839fadb8f45d8a7a2783466faedd03373366c23
\ No newline at end of file
diff --git a/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/OpenSearchReactorNetty4IntegTestCase.java b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/OpenSearchReactorNetty4IntegTestCase.java
new file mode 100644
index 0000000000000..abbd50bf1b235
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/OpenSearchReactorNetty4IntegTestCase.java
@@ -0,0 +1,73 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch;
+
+import org.opensearch.common.network.NetworkModule;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.transport.Netty4ModulePlugin;
+import org.opensearch.transport.reactor.ReactorNetty4Plugin;
+import org.opensearch.transport.reactor.netty4.ReactorNetty4Transport;
+
+import java.util.Collection;
+import java.util.List;
+
+public abstract class OpenSearchReactorNetty4IntegTestCase extends OpenSearchIntegTestCase {
+
+ @Override
+ protected boolean ignoreExternalCluster() {
+ return true;
+ }
+
+ @Override
+ protected boolean addMockTransportService() {
+ return false;
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal));
+ // randomize netty settings
+ if (randomBoolean()) {
+ builder.put(ReactorNetty4Transport.SETTING_WORKER_COUNT.getKey(), random().nextInt(3) + 1);
+ }
+ builder.put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4ModulePlugin.NETTY_TRANSPORT_NAME);
+ builder.put(NetworkModule.HTTP_TYPE_KEY, ReactorNetty4Plugin.REACTOR_NETTY_HTTP_TRANSPORT_NAME);
+ return builder.build();
+ }
+
+ @Override
+ protected Collection> nodePlugins() {
+ return List.of(ReactorNetty4Plugin.class, Netty4ModulePlugin.class);
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequestSizeLimitIT.java b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequestSizeLimitIT.java
new file mode 100644
index 0000000000000..833d60375a2bd
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequestSizeLimitIT.java
@@ -0,0 +1,159 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.OpenSearchReactorNetty4IntegTestCase;
+import org.opensearch.common.collect.Tuple;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.common.transport.TransportAddress;
+import org.opensearch.core.common.unit.ByteSizeUnit;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.indices.breaker.HierarchyCircuitBreakerService;
+import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope;
+import org.opensearch.test.OpenSearchIntegTestCase.Scope;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.util.ReferenceCounted;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasSize;
+
+/**
+ * This test checks that in-flight requests are limited on HTTP level and that requests that are excluded from limiting can pass.
+ *
+ * As the same setting is also used to limit in-flight requests on transport level, we avoid transport messages by forcing
+ * a single node "cluster".
+ */
+@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numClientNodes = 0, numDataNodes = 1)
+public class ReactorNetty4HttpRequestSizeLimitIT extends OpenSearchReactorNetty4IntegTestCase {
+
+ private static final ByteSizeValue LIMIT = new ByteSizeValue(2, ByteSizeUnit.KB);
+
+ @Override
+ protected boolean addMockHttpTransport() {
+ return false; // enable http
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), LIMIT)
+ .build();
+ }
+
+ public void testLimitsInFlightRequests() throws Exception {
+ ensureGreen();
+
+ // we use the limit size as a (very) rough indication on how many requests we should sent to hit the limit
+ int numRequests = LIMIT.bytesAsInt() / 100;
+
+ StringBuilder bulkRequest = new StringBuilder();
+ for (int i = 0; i < numRequests; i++) {
+ bulkRequest.append("{\"index\": {}}");
+ bulkRequest.append(System.lineSeparator());
+ bulkRequest.append("{ \"field\" : \"value\" }");
+ bulkRequest.append(System.lineSeparator());
+ }
+
+ List> requests = new ArrayList<>();
+ for (int i = 0; i < 150; i++) {
+ requests.add(Tuple.tuple("/index/_bulk", bulkRequest));
+ }
+
+ HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class);
+ TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient nettyHttpClient = ReactorHttpClient.create(false)) {
+ final Collection singleResponse = nettyHttpClient.post(transportAddress.address(), requests.subList(0, 1));
+ try {
+ assertThat(singleResponse, hasSize(1));
+ assertAtLeastOnceExpectedStatus(singleResponse, HttpResponseStatus.OK);
+
+ final Collection multipleResponses = nettyHttpClient.post(transportAddress.address(), requests);
+ try {
+ assertThat(multipleResponses, hasSize(requests.size()));
+ assertAtLeastOnceExpectedStatus(multipleResponses, HttpResponseStatus.TOO_MANY_REQUESTS);
+ } finally {
+ multipleResponses.forEach(ReferenceCounted::release);
+ }
+ } finally {
+ singleResponse.forEach(ReferenceCounted::release);
+ }
+ }
+ }
+
+ public void testDoesNotLimitExcludedRequests() throws Exception {
+ ensureGreen();
+
+ List> requestUris = new ArrayList<>();
+ for (int i = 0; i < 1500; i++) {
+ requestUris.add(Tuple.tuple("/_cluster/settings", "{ \"transient\": {\"search.default_search_timeout\": \"40s\" } }"));
+ }
+
+ HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class);
+ TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient nettyHttpClient = ReactorHttpClient.create(false)) {
+ final Collection responses = nettyHttpClient.put(transportAddress.address(), requestUris);
+ try {
+ assertThat(responses, hasSize(requestUris.size()));
+ assertAllInExpectedStatus(responses, HttpResponseStatus.OK);
+ } finally {
+ responses.forEach(ReferenceCounted::release);
+ }
+ }
+ }
+
+ private void assertAtLeastOnceExpectedStatus(Collection responses, HttpResponseStatus expectedStatus) {
+ long countExpectedStatus = responses.stream().filter(r -> r.status().equals(expectedStatus)).count();
+ assertThat("Expected at least one request with status [" + expectedStatus + "]", countExpectedStatus, greaterThan(0L));
+ }
+
+ private void assertAllInExpectedStatus(Collection responses, HttpResponseStatus expectedStatus) {
+ long countUnexpectedStatus = responses.stream().filter(r -> r.status().equals(expectedStatus) == false).count();
+ assertThat(
+ "Expected all requests with status [" + expectedStatus + "] but [" + countUnexpectedStatus + "] requests had a different one",
+ countUnexpectedStatus,
+ equalTo(0L)
+ );
+ }
+
+}
diff --git a/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4PipeliningIT.java b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4PipeliningIT.java
new file mode 100644
index 0000000000000..c0e43de06f6ff
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/internalClusterTest/java/org/opensearch/http/reactor/netty4/ReactorNetty4PipeliningIT.java
@@ -0,0 +1,68 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.OpenSearchReactorNetty4IntegTestCase;
+import org.opensearch.core.common.transport.TransportAddress;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope;
+import org.opensearch.test.OpenSearchIntegTestCase.Scope;
+
+import java.util.Collection;
+import java.util.Locale;
+
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.util.ReferenceCounted;
+
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+
+@ClusterScope(scope = Scope.TEST, supportsDedicatedMasters = false, numDataNodes = 1)
+public class ReactorNetty4PipeliningIT extends OpenSearchReactorNetty4IntegTestCase {
+
+ @Override
+ protected boolean addMockHttpTransport() {
+ return false; // enable http
+ }
+
+ public void testThatNettyHttpServerSupportsPipelining() throws Exception {
+ String[] requests = new String[] { "/", "/_nodes/stats", "/", "/_cluster/state", "/" };
+
+ HttpServerTransport httpServerTransport = internalCluster().getInstance(HttpServerTransport.class);
+ TransportAddress[] boundAddresses = httpServerTransport.boundAddress().boundAddresses();
+ TransportAddress transportAddress = randomFrom(boundAddresses);
+
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ Collection responses = client.get(transportAddress.address(), true, requests);
+ try {
+ assertThat(responses, hasSize(5));
+
+ Collection opaqueIds = ReactorHttpClient.returnOpaqueIds(responses);
+ assertOpaqueIdsInOrder(opaqueIds);
+ } finally {
+ responses.forEach(ReferenceCounted::release);
+ }
+ }
+ }
+
+ private void assertOpaqueIdsInOrder(Collection opaqueIds) {
+ // check if opaque ids are monotonically increasing
+ int i = 0;
+ String msg = String.format(Locale.ROOT, "Expected list of opaque ids to be monotonically increasing, got [%s]", opaqueIds);
+ for (String opaqueId : opaqueIds) {
+ assertThat(msg, opaqueId, is(String.valueOf(i++)));
+ }
+ }
+
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/HttpConversionUtil.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/HttpConversionUtil.java
new file mode 100644
index 0000000000000..bd75227dabd08
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/HttpConversionUtil.java
@@ -0,0 +1,47 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.rest.RestRequest;
+
+import io.netty.handler.codec.http.HttpMethod;
+
+final class HttpConversionUtil {
+ private HttpConversionUtil() {}
+
+ /**
+ * Converts {@link HttpMethod} to {@link RestRequest.Method}
+ * @param method {@link HttpMethod} method
+ * @return corresponding {@link RestRequest.Method}
+ * @throws IllegalArgumentException if HTTP method is not supported
+ */
+ public static RestRequest.Method convertMethod(HttpMethod method) {
+ if (method == HttpMethod.GET) {
+ return RestRequest.Method.GET;
+ } else if (method == HttpMethod.POST) {
+ return RestRequest.Method.POST;
+ } else if (method == HttpMethod.PUT) {
+ return RestRequest.Method.PUT;
+ } else if (method == HttpMethod.DELETE) {
+ return RestRequest.Method.DELETE;
+ } else if (method == HttpMethod.HEAD) {
+ return RestRequest.Method.HEAD;
+ } else if (method == HttpMethod.OPTIONS) {
+ return RestRequest.Method.OPTIONS;
+ } else if (method == HttpMethod.PATCH) {
+ return RestRequest.Method.PATCH;
+ } else if (method == HttpMethod.TRACE) {
+ return RestRequest.Method.TRACE;
+ } else if (method == HttpMethod.CONNECT) {
+ return RestRequest.Method.CONNECT;
+ } else {
+ throw new IllegalArgumentException("Unexpected http method: " + method);
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingHttpChannel.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingHttpChannel.java
new file mode 100644
index 0000000000000..98b359319ff1b
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingHttpChannel.java
@@ -0,0 +1,76 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.common.concurrent.CompletableContext;
+import org.opensearch.core.action.ActionListener;
+import org.opensearch.http.HttpChannel;
+import org.opensearch.http.HttpResponse;
+import org.opensearch.transport.reactor.netty4.Netty4Utils;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import reactor.core.publisher.FluxSink;
+import reactor.netty.http.server.HttpServerRequest;
+import reactor.netty.http.server.HttpServerResponse;
+
+class NonStreamingHttpChannel implements HttpChannel {
+ private final HttpServerRequest request;
+ private final HttpServerResponse response;
+ private final CompletableContext closeContext = new CompletableContext<>();
+ private final FluxSink emitter;
+
+ NonStreamingHttpChannel(HttpServerRequest request, HttpServerResponse response, FluxSink emitter) {
+ this.request = request;
+ this.response = response;
+ this.emitter = emitter;
+ this.request.withConnection(connection -> Netty4Utils.addListener(connection.channel().closeFuture(), closeContext));
+ }
+
+ @Override
+ public boolean isOpen() {
+ final AtomicBoolean isOpen = new AtomicBoolean();
+ request.withConnection(connection -> isOpen.set(connection.channel().isOpen()));
+ return isOpen.get();
+ }
+
+ @Override
+ public void close() {
+ request.withConnection(connection -> connection.channel().close());
+ }
+
+ @Override
+ public void addCloseListener(ActionListener listener) {
+ closeContext.addListener(ActionListener.toBiConsumer(listener));
+ }
+
+ @Override
+ public void sendResponse(HttpResponse response, ActionListener listener) {
+ emitter.next(createResponse(response));
+ listener.onResponse(null);
+ emitter.complete();
+ }
+
+ @Override
+ public InetSocketAddress getRemoteAddress() {
+ return (InetSocketAddress) response.remoteAddress();
+ }
+
+ @Override
+ public InetSocketAddress getLocalAddress() {
+ return (InetSocketAddress) response.hostAddress();
+ }
+
+ FullHttpResponse createResponse(HttpResponse response) {
+ return (FullHttpResponse) response;
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingRequestConsumer.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingRequestConsumer.java
new file mode 100644
index 0000000000000..d43e23e800e65
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/NonStreamingRequestConsumer.java
@@ -0,0 +1,104 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.http.AbstractHttpServerTransport;
+import org.opensearch.http.HttpRequest;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.LastHttpContent;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import reactor.core.Disposable;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+import reactor.netty.http.server.HttpServerRequest;
+import reactor.netty.http.server.HttpServerResponse;
+
+class NonStreamingRequestConsumer implements Consumer, Publisher, Disposable {
+ private final HttpServerRequest request;
+ private final HttpServerResponse response;
+ private final CompositeByteBuf content;
+ private final Publisher publisher;
+ private final AbstractHttpServerTransport transport;
+ private final AtomicBoolean disposed = new AtomicBoolean(false);
+ private volatile FluxSink emitter;
+
+ NonStreamingRequestConsumer(
+ AbstractHttpServerTransport transport,
+ HttpServerRequest request,
+ HttpServerResponse response,
+ int maxCompositeBufferComponents
+ ) {
+ this.transport = transport;
+ this.request = request;
+ this.response = response;
+ this.content = response.alloc().compositeBuffer(maxCompositeBufferComponents);
+ this.publisher = Flux.create(emitter -> register(emitter));
+ }
+
+ private void register(FluxSink emitter) {
+ this.emitter = emitter.onDispose(this).onCancel(this);
+ }
+
+ @Override
+ public void accept(T message) {
+ try {
+ if (message instanceof LastHttpContent) {
+ process(message, emitter);
+ } else if (message instanceof HttpContent) {
+ process(message, emitter);
+ }
+ } catch (Throwable ex) {
+ emitter.error(ex);
+ }
+ }
+
+ public void process(HttpContent in, FluxSink emitter) {
+ // Consume request body in full before dispatching it
+ content.addComponent(true, in.content().retain());
+
+ if (in instanceof LastHttpContent) {
+ final NonStreamingHttpChannel channel = new NonStreamingHttpChannel(request, response, emitter);
+ final HttpRequest r = createRequest(request, content);
+
+ try {
+ transport.incomingRequest(r, channel);
+ } catch (Exception ex) {
+ emitter.error(ex);
+ transport.onException(channel, ex);
+ } finally {
+ r.release();
+ if (disposed.compareAndSet(false, true)) {
+ this.content.release();
+ }
+ }
+ }
+ }
+
+ HttpRequest createRequest(HttpServerRequest request, CompositeByteBuf content) {
+ return new ReactorNetty4HttpRequest(request, content.retain());
+ }
+
+ @Override
+ public void subscribe(Subscriber super HttpContent> s) {
+ publisher.subscribe(s);
+ }
+
+ @Override
+ public void dispose() {
+ if (disposed.compareAndSet(false, true)) {
+ this.content.release();
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequest.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequest.java
new file mode 100644
index 0000000000000..4406c555a5b04
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpRequest.java
@@ -0,0 +1,272 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.core.common.bytes.BytesReference;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.http.HttpRequest;
+import org.opensearch.rest.RestRequest;
+import org.opensearch.transport.reactor.netty4.Netty4Utils;
+
+import java.util.AbstractMap;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.DefaultHttpHeaders;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.cookie.Cookie;
+import io.netty.handler.codec.http.cookie.ServerCookieDecoder;
+import io.netty.handler.codec.http.cookie.ServerCookieEncoder;
+import reactor.netty.http.server.HttpServerRequest;
+
+class ReactorNetty4HttpRequest implements HttpRequest {
+ private final String protocol;
+ private final HttpMethod method;
+ private final String uri;
+ private final ByteBuf content;
+ private final HttpHeadersMap headers;
+ private final AtomicBoolean released;
+ private final Exception inboundException;
+ private final boolean pooled;
+
+ ReactorNetty4HttpRequest(HttpServerRequest request, ByteBuf content) {
+ this(request, new HttpHeadersMap(request.requestHeaders()), new AtomicBoolean(false), true, content);
+ }
+
+ ReactorNetty4HttpRequest(HttpServerRequest request, ByteBuf content, Exception inboundException) {
+ this(
+ request.protocol(),
+ request.method(),
+ request.uri(),
+ new HttpHeadersMap(request.requestHeaders()),
+ new AtomicBoolean(false),
+ true,
+ content,
+ inboundException
+ );
+ }
+
+ private ReactorNetty4HttpRequest(
+ HttpServerRequest request,
+ HttpHeadersMap headers,
+ AtomicBoolean released,
+ boolean pooled,
+ ByteBuf content
+ ) {
+ this(request.protocol(), request.method(), request.uri(), headers, released, pooled, content, null);
+ }
+
+ private ReactorNetty4HttpRequest(
+ String protocol,
+ HttpMethod method,
+ String uri,
+ HttpHeadersMap headers,
+ AtomicBoolean released,
+ boolean pooled,
+ ByteBuf content,
+ Exception inboundException
+ ) {
+
+ this.protocol = protocol;
+ this.method = method;
+ this.uri = uri;
+ this.headers = headers;
+ this.content = content;
+ this.pooled = pooled;
+ this.released = released;
+ this.inboundException = inboundException;
+ }
+
+ @Override
+ public RestRequest.Method method() {
+ return HttpConversionUtil.convertMethod(method);
+ }
+
+ @Override
+ public String uri() {
+ return uri;
+ }
+
+ @Override
+ public BytesReference content() {
+ assert released.get() == false;
+ return Netty4Utils.toBytesReference(content);
+ }
+
+ @Override
+ public void release() {
+ if (pooled && released.compareAndSet(false, true)) {
+ content.release();
+ }
+ }
+
+ @Override
+ public HttpRequest releaseAndCopy() {
+ assert released.get() == false;
+ if (pooled == false) {
+ return this;
+ }
+ try {
+ final ByteBuf copiedContent = Unpooled.copiedBuffer(content);
+ return new ReactorNetty4HttpRequest(protocol, method, uri, headers, new AtomicBoolean(false), false, copiedContent, null);
+ } finally {
+ release();
+ }
+ }
+
+ @Override
+ public final Map> getHeaders() {
+ return headers;
+ }
+
+ @Override
+ public List strictCookies() {
+ String cookieString = headers.httpHeaders.get(HttpHeaderNames.COOKIE);
+ if (cookieString != null) {
+ Set cookies = ServerCookieDecoder.STRICT.decode(cookieString);
+ if (!cookies.isEmpty()) {
+ return ServerCookieEncoder.STRICT.encode(cookies);
+ }
+ }
+ return Collections.emptyList();
+ }
+
+ @Override
+ public HttpVersion protocolVersion() {
+ if (protocol.equals(io.netty.handler.codec.http.HttpVersion.HTTP_1_0.toString())) {
+ return HttpRequest.HttpVersion.HTTP_1_0;
+ } else if (protocol.equals(io.netty.handler.codec.http.HttpVersion.HTTP_1_1.toString())) {
+ return HttpRequest.HttpVersion.HTTP_1_1;
+ } else {
+ throw new IllegalArgumentException("Unexpected http protocol version: " + protocol);
+ }
+ }
+
+ @Override
+ public HttpRequest removeHeader(String header) {
+ HttpHeaders headersWithoutContentTypeHeader = new DefaultHttpHeaders();
+ headersWithoutContentTypeHeader.add(headers.httpHeaders);
+ headersWithoutContentTypeHeader.remove(header);
+
+ return new ReactorNetty4HttpRequest(
+ protocol,
+ method,
+ uri,
+ new HttpHeadersMap(headersWithoutContentTypeHeader),
+ released,
+ pooled,
+ content,
+ null
+ );
+ }
+
+ @Override
+ public ReactorNetty4HttpResponse createResponse(RestStatus status, BytesReference content) {
+ return new ReactorNetty4HttpResponse(
+ headers.httpHeaders,
+ io.netty.handler.codec.http.HttpVersion.valueOf(protocol),
+ status,
+ content
+ );
+ }
+
+ @Override
+ public Exception getInboundException() {
+ return inboundException;
+ }
+
+ /**
+ * A wrapper of {@link HttpHeaders} that implements a map to prevent copying unnecessarily. This class does not support modifications
+ * and due to the underlying implementation, it performs case insensitive lookups of key to values.
+ *
+ * It is important to note that this implementation does have some downsides in that each invocation of the
+ * {@link #values()} and {@link #entrySet()} methods will perform a copy of the values in the HttpHeaders rather than returning a
+ * view of the underlying values.
+ */
+ private static class HttpHeadersMap implements Map> {
+
+ private final HttpHeaders httpHeaders;
+
+ private HttpHeadersMap(HttpHeaders httpHeaders) {
+ this.httpHeaders = httpHeaders;
+ }
+
+ @Override
+ public int size() {
+ return httpHeaders.size();
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return httpHeaders.isEmpty();
+ }
+
+ @Override
+ public boolean containsKey(Object key) {
+ return key instanceof String && httpHeaders.contains((String) key);
+ }
+
+ @Override
+ public boolean containsValue(Object value) {
+ return value instanceof List && httpHeaders.names().stream().map(httpHeaders::getAll).anyMatch(value::equals);
+ }
+
+ @Override
+ public List get(Object key) {
+ return key instanceof String ? httpHeaders.getAll((String) key) : null;
+ }
+
+ @Override
+ public List put(String key, List value) {
+ throw new UnsupportedOperationException("modifications are not supported");
+ }
+
+ @Override
+ public List remove(Object key) {
+ throw new UnsupportedOperationException("modifications are not supported");
+ }
+
+ @Override
+ public void putAll(Map extends String, ? extends List> m) {
+ throw new UnsupportedOperationException("modifications are not supported");
+ }
+
+ @Override
+ public void clear() {
+ throw new UnsupportedOperationException("modifications are not supported");
+ }
+
+ @Override
+ public Set keySet() {
+ return httpHeaders.names();
+ }
+
+ @Override
+ public Collection> values() {
+ return httpHeaders.names().stream().map(k -> Collections.unmodifiableList(httpHeaders.getAll(k))).collect(Collectors.toList());
+ }
+
+ @Override
+ public Set>> entrySet() {
+ return httpHeaders.names()
+ .stream()
+ .map(k -> new AbstractMap.SimpleImmutableEntry<>(k, httpHeaders.getAll(k)))
+ .collect(Collectors.toSet());
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpResponse.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpResponse.java
new file mode 100644
index 0000000000000..c45ad54b668a3
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpResponse.java
@@ -0,0 +1,42 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.core.common.bytes.BytesReference;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.http.HttpResponse;
+import org.opensearch.transport.reactor.netty4.Netty4Utils;
+
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+
+class ReactorNetty4HttpResponse extends DefaultFullHttpResponse implements HttpResponse {
+ private final HttpHeaders requestHeaders;
+
+ ReactorNetty4HttpResponse(HttpHeaders requestHeaders, HttpVersion version, RestStatus status, BytesReference content) {
+ super(version, HttpResponseStatus.valueOf(status.getStatus()), Netty4Utils.toByteBuf(content));
+ this.requestHeaders = requestHeaders;
+ }
+
+ @Override
+ public void addHeader(String name, String value) {
+ headers().add(name, value);
+ }
+
+ @Override
+ public boolean containsHeader(String name) {
+ return headers().contains(name);
+ }
+
+ public HttpHeaders requestHeaders() {
+ return requestHeaders;
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerChannel.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerChannel.java
new file mode 100644
index 0000000000000..84360bf028ba9
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerChannel.java
@@ -0,0 +1,53 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.common.concurrent.CompletableContext;
+import org.opensearch.core.action.ActionListener;
+import org.opensearch.http.HttpServerChannel;
+import org.opensearch.transport.reactor.netty4.Netty4Utils;
+
+import java.net.InetSocketAddress;
+
+import io.netty.channel.Channel;
+
+class ReactorNetty4HttpServerChannel implements HttpServerChannel {
+ private final Channel channel;
+ private final CompletableContext closeContext = new CompletableContext<>();
+
+ ReactorNetty4HttpServerChannel(Channel channel) {
+ this.channel = channel;
+ Netty4Utils.addListener(this.channel.closeFuture(), closeContext);
+ }
+
+ @Override
+ public InetSocketAddress getLocalAddress() {
+ return (InetSocketAddress) channel.localAddress();
+ }
+
+ @Override
+ public void addCloseListener(ActionListener listener) {
+ closeContext.addListener(ActionListener.toBiConsumer(listener));
+ }
+
+ @Override
+ public boolean isOpen() {
+ return channel.isOpen();
+ }
+
+ @Override
+ public void close() {
+ channel.close();
+ }
+
+ @Override
+ public String toString() {
+ return "ReactorNetty4HttpChannel{localAddress=" + getLocalAddress() + "}";
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransport.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransport.java
new file mode 100644
index 0000000000000..d4a5a9ad83af6
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransport.java
@@ -0,0 +1,313 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.common.network.NetworkService;
+import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Setting.Property;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.BigArrays;
+import org.opensearch.common.util.concurrent.OpenSearchExecutors;
+import org.opensearch.common.util.io.IOUtils;
+import org.opensearch.common.util.net.NetUtils;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.core.xcontent.NamedXContentRegistry;
+import org.opensearch.http.AbstractHttpServerTransport;
+import org.opensearch.http.HttpChannel;
+import org.opensearch.http.HttpReadTimeoutException;
+import org.opensearch.http.HttpServerChannel;
+import org.opensearch.telemetry.tracing.Tracer;
+import org.opensearch.threadpool.ThreadPool;
+import org.opensearch.transport.reactor.SharedGroupFactory;
+import org.opensearch.transport.reactor.netty4.Netty4Utils;
+
+import java.net.InetSocketAddress;
+import java.net.SocketOption;
+import java.time.Duration;
+
+import io.netty.channel.ChannelOption;
+import io.netty.channel.socket.nio.NioChannelOption;
+import io.netty.handler.codec.http.DefaultLastHttpContent;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.timeout.ReadTimeoutException;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Mono;
+import reactor.core.scheduler.Scheduler;
+import reactor.core.scheduler.Schedulers;
+import reactor.netty.DisposableServer;
+import reactor.netty.http.HttpProtocol;
+import reactor.netty.http.server.HttpServer;
+import reactor.netty.http.server.HttpServerRequest;
+import reactor.netty.http.server.HttpServerResponse;
+
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_CONNECT_TIMEOUT;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_COUNT;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_IDLE;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_KEEP_INTERVAL;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS;
+import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE;
+
+/**
+ * The HTTP transport implementations based on Reactor Netty (see please {@link HttpServer}).
+ */
+public class ReactorNetty4HttpServerTransport extends AbstractHttpServerTransport {
+ private static final String SETTING_KEY_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS = "http.netty.max_composite_buffer_components";
+ private static final ByteSizeValue MTU = new ByteSizeValue(Long.parseLong(System.getProperty("opensearch.net.mtu", "1500")));
+
+ /**
+ * The number of Reactor Netty HTTP workers
+ */
+ public static final Setting SETTING_HTTP_WORKER_COUNT = Setting.intSetting("http.netty.worker_count", 0, Property.NodeScope);
+
+ /**
+ * The maximum number of composite components for request accumulation
+ */
+ public static Setting SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS = new Setting<>(
+ SETTING_KEY_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS,
+ (s) -> {
+ ByteSizeValue maxContentLength = SETTING_HTTP_MAX_CONTENT_LENGTH.get(s);
+ /*
+ * Netty accumulates buffers containing data from all incoming network packets that make up one HTTP request in an instance of
+ * io.netty.buffer.CompositeByteBuf (think of it as a buffer of buffers). Once its capacity is reached, the buffer will iterate
+ * over its individual entries and put them into larger buffers (see io.netty.buffer.CompositeByteBuf#consolidateIfNeeded()
+ * for implementation details). We want to to resize that buffer because this leads to additional garbage on the heap and also
+ * increases the application's native memory footprint (as direct byte buffers hold their contents off-heap).
+ *
+ * With this setting we control the CompositeByteBuf's capacity (which is by default 1024, see
+ * io.netty.handler.codec.MessageAggregator#DEFAULT_MAX_COMPOSITEBUFFER_COMPONENTS). To determine a proper default capacity for
+ * that buffer, we need to consider that the upper bound for the size of HTTP requests is determined by `maxContentLength`. The
+ * number of buffers that are needed depend on how often Netty reads network packets which depends on the network type (MTU).
+ * We assume here that OpenSearch receives HTTP requests via an Ethernet connection which has a MTU of 1500 bytes.
+ *
+ * Note that we are *not* pre-allocating any memory based on this setting but rather determine the CompositeByteBuf's capacity.
+ * The tradeoff is between less (but larger) buffers that are contained in the CompositeByteBuf and more (but smaller) buffers.
+ * With the default max content length of 100MB and a MTU of 1500 bytes we would allow 69905 entries.
+ */
+ long maxBufferComponentsEstimate = Math.round((double) (maxContentLength.getBytes() / MTU.getBytes()));
+ // clamp value to the allowed range
+ long maxBufferComponents = Math.max(2, Math.min(maxBufferComponentsEstimate, Integer.MAX_VALUE));
+ return String.valueOf(maxBufferComponents);
+ // Netty's CompositeByteBuf implementation does not allow less than two components.
+ },
+ s -> Setting.parseInt(s, 2, Integer.MAX_VALUE, SETTING_KEY_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS),
+ Property.NodeScope
+ );
+
+ private final SharedGroupFactory sharedGroupFactory;
+ private final int readTimeoutMillis;
+ private final int connectTimeoutMillis;
+ private final int maxCompositeBufferComponents;
+ private final ByteSizeValue maxInitialLineLength;
+ private final ByteSizeValue maxHeaderSize;
+ private final ByteSizeValue maxChunkSize;
+ private volatile SharedGroupFactory.SharedGroup sharedGroup;
+ private volatile DisposableServer disposableServer;
+ private volatile Scheduler scheduler;
+
+ /**
+ * Creates new HTTP transport implementations based on Reactor Netty (see please {@link HttpServer}).
+ * @param settings settings
+ * @param networkService network service
+ * @param bigArrays big array allocator
+ * @param threadPool thread pool instance
+ * @param xContentRegistry XContent registry instance
+ * @param dispatcher dispatcher instance
+ * @param clusterSettings cluster settings
+ * @param sharedGroupFactory shared group factory
+ * @param tracer tracer instance
+ */
+ public ReactorNetty4HttpServerTransport(
+ Settings settings,
+ NetworkService networkService,
+ BigArrays bigArrays,
+ ThreadPool threadPool,
+ NamedXContentRegistry xContentRegistry,
+ Dispatcher dispatcher,
+ ClusterSettings clusterSettings,
+ SharedGroupFactory sharedGroupFactory,
+ Tracer tracer
+ ) {
+ super(settings, networkService, bigArrays, threadPool, xContentRegistry, dispatcher, clusterSettings, tracer);
+ Netty4Utils.setAvailableProcessors(OpenSearchExecutors.NODE_PROCESSORS_SETTING.get(settings));
+ this.readTimeoutMillis = Math.toIntExact(SETTING_HTTP_READ_TIMEOUT.get(settings).getMillis());
+ this.connectTimeoutMillis = Math.toIntExact(SETTING_HTTP_CONNECT_TIMEOUT.get(settings).getMillis());
+ this.sharedGroupFactory = sharedGroupFactory;
+ this.maxCompositeBufferComponents = SETTING_HTTP_NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS.get(settings);
+ this.maxChunkSize = SETTING_HTTP_MAX_CHUNK_SIZE.get(settings);
+ this.maxHeaderSize = SETTING_HTTP_MAX_HEADER_SIZE.get(settings);
+ this.maxInitialLineLength = SETTING_HTTP_MAX_INITIAL_LINE_LENGTH.get(settings);
+ }
+
+ /**
+ * Binds the transport engine to the socket address
+ * @param socketAddress socket address to bind to
+ */
+ @Override
+ protected HttpServerChannel bind(InetSocketAddress socketAddress) throws Exception {
+ final HttpServer server = configureChannelOptions(
+ HttpServer.create()
+ .httpFormDecoder(builder -> builder.scheduler(scheduler))
+ .idleTimeout(Duration.ofMillis(connectTimeoutMillis))
+ .readTimeout(Duration.ofMillis(readTimeoutMillis))
+ .runOn(sharedGroup.getLowLevelGroup())
+ .bindAddress(() -> socketAddress)
+ .compress(true)
+ .httpRequestDecoder(
+ spec -> spec.maxChunkSize(maxChunkSize.bytesAsInt())
+ .maxHeaderSize(maxHeaderSize.bytesAsInt())
+ .maxInitialLineLength(maxInitialLineLength.bytesAsInt())
+ )
+ .protocol(HttpProtocol.HTTP11, HttpProtocol.H2C)
+ .handle((req, res) -> incomingRequest(req, res))
+ );
+
+ disposableServer = server.bindNow();
+ return new ReactorNetty4HttpServerChannel(disposableServer.channel());
+ }
+
+ private HttpServer configureChannelOptions(final HttpServer server1) {
+ HttpServer configured = server1.childOption(ChannelOption.TCP_NODELAY, SETTING_HTTP_TCP_NO_DELAY.get(settings))
+ .childOption(ChannelOption.SO_KEEPALIVE, SETTING_HTTP_TCP_KEEP_ALIVE.get(settings));
+
+ if (SETTING_HTTP_TCP_KEEP_ALIVE.get(settings)) {
+ // Netty logs a warning if it can't set the option, so try this only on supported platforms
+ if (IOUtils.LINUX || IOUtils.MAC_OS_X) {
+ if (SETTING_HTTP_TCP_KEEP_IDLE.get(settings) >= 0) {
+ final SocketOption keepIdleOption = NetUtils.getTcpKeepIdleSocketOptionOrNull();
+ if (keepIdleOption != null) {
+ configured = configured.childOption(NioChannelOption.of(keepIdleOption), SETTING_HTTP_TCP_KEEP_IDLE.get(settings));
+ }
+ }
+ if (SETTING_HTTP_TCP_KEEP_INTERVAL.get(settings) >= 0) {
+ final SocketOption keepIntervalOption = NetUtils.getTcpKeepIntervalSocketOptionOrNull();
+ if (keepIntervalOption != null) {
+ configured = configured.childOption(
+ NioChannelOption.of(keepIntervalOption),
+ SETTING_HTTP_TCP_KEEP_INTERVAL.get(settings)
+ );
+ }
+ }
+ if (SETTING_HTTP_TCP_KEEP_COUNT.get(settings) >= 0) {
+ final SocketOption keepCountOption = NetUtils.getTcpKeepCountSocketOptionOrNull();
+ if (keepCountOption != null) {
+ configured = configured.childOption(
+ NioChannelOption.of(keepCountOption),
+ SETTING_HTTP_TCP_KEEP_COUNT.get(settings)
+ );
+ }
+ }
+ }
+ }
+
+ final ByteSizeValue tcpSendBufferSize = SETTING_HTTP_TCP_SEND_BUFFER_SIZE.get(settings);
+ if (tcpSendBufferSize.getBytes() > 0) {
+ configured = configured.childOption(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.getBytes()));
+ }
+
+ final ByteSizeValue tcpReceiveBufferSize = SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE.get(settings);
+ if (tcpReceiveBufferSize.getBytes() > 0) {
+ configured = configured.childOption(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.getBytes()));
+ }
+
+ final boolean reuseAddress = SETTING_HTTP_TCP_REUSE_ADDRESS.get(settings);
+ configured = configured.option(ChannelOption.SO_REUSEADDR, reuseAddress);
+ configured = configured.childOption(ChannelOption.SO_REUSEADDR, reuseAddress);
+
+ return configured;
+ }
+
+ /**
+ * Handles incoming Reactor Netty request
+ * @param request request instance
+ * @param response response instances
+ * @return response publisher
+ */
+ protected Publisher incomingRequest(HttpServerRequest request, HttpServerResponse response) {
+ final NonStreamingRequestConsumer consumer = new NonStreamingRequestConsumer<>(
+ this,
+ request,
+ response,
+ maxCompositeBufferComponents
+ );
+
+ request.receiveContent().switchIfEmpty(Mono.just(DefaultLastHttpContent.EMPTY_LAST_CONTENT)).subscribe(consumer);
+
+ return Mono.from(consumer).flatMap(hc -> {
+ final FullHttpResponse r = (FullHttpResponse) hc;
+ response.status(r.status());
+ response.trailerHeaders(c -> r.trailingHeaders().forEach(h -> c.add(h.getKey(), h.getValue())));
+ response.chunkedTransfer(false);
+ response.compression(true);
+ r.headers().forEach(h -> response.addHeader(h.getKey(), h.getValue()));
+ return Mono.from(response.sendObject(r.content()));
+ });
+ }
+
+ /**
+ * Called to tear down internal resources
+ */
+ @Override
+ protected void stopInternal() {
+ if (sharedGroup != null) {
+ sharedGroup.shutdown();
+ sharedGroup = null;
+ }
+
+ if (scheduler != null) {
+ scheduler.dispose();
+ scheduler = null;
+ }
+
+ if (disposableServer != null) {
+ disposableServer.disposeNow();
+ disposableServer = null;
+ }
+ }
+
+ /**
+ * Starts the transport
+ */
+ @Override
+ protected void doStart() {
+ boolean success = false;
+ try {
+ scheduler = Schedulers.newBoundedElastic(
+ Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE,
+ Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE,
+ "http-form-decoder"
+ );
+ sharedGroup = sharedGroupFactory.getHttpGroup();
+ bindServer();
+ success = true;
+ } finally {
+ if (success == false) {
+ doStop(); // otherwise we leak threads since we never moved to started
+ }
+ }
+ }
+
+ @Override
+ public void onException(HttpChannel channel, Exception cause) {
+ if (cause instanceof ReadTimeoutException) {
+ super.onException(channel, new HttpReadTimeoutException(readTimeoutMillis, cause));
+ } else {
+ super.onException(channel, cause);
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/package-info.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/package-info.java
new file mode 100644
index 0000000000000..b5ecb0b62f79d
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/http/reactor/netty4/package-info.java
@@ -0,0 +1,12 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/**
+ * The new HTTP transport implementations based on Reactor Netty.
+ */
+package org.opensearch.http.reactor.netty4;
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/ReactorNetty4Plugin.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/ReactorNetty4Plugin.java
new file mode 100644
index 0000000000000..dc310c3793109
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/ReactorNetty4Plugin.java
@@ -0,0 +1,109 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+package org.opensearch.transport.reactor;
+
+import org.opensearch.common.SetOnce;
+import org.opensearch.common.network.NetworkService;
+import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.BigArrays;
+import org.opensearch.common.util.PageCacheRecycler;
+import org.opensearch.core.indices.breaker.CircuitBreakerService;
+import org.opensearch.core.xcontent.NamedXContentRegistry;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.http.reactor.netty4.ReactorNetty4HttpServerTransport;
+import org.opensearch.plugins.NetworkPlugin;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.telemetry.tracing.Tracer;
+import org.opensearch.threadpool.ThreadPool;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+
+/**
+ * The experimental network plugin that introduces new transport implementations based on Reactor Netty.
+ */
+public class ReactorNetty4Plugin extends Plugin implements NetworkPlugin {
+ /**
+ * The name of new experimental HTTP transport implementations based on Reactor Netty.
+ */
+ public static final String REACTOR_NETTY_HTTP_TRANSPORT_NAME = "reactor-netty4";
+
+ private final SetOnce groupFactory = new SetOnce<>();
+
+ /**
+ * Default constructor
+ */
+ public ReactorNetty4Plugin() {}
+
+ /**
+ * Returns a list of additional {@link Setting} definitions for this plugin.
+ */
+ @Override
+ public List> getSettings() {
+ return Arrays.asList(/* no setting registered since we're picking the onces from Netty 4 transport */);
+ }
+
+ /**
+ * Returns a map of {@link HttpServerTransport} suppliers.
+ * See {@link org.opensearch.common.network.NetworkModule#HTTP_TYPE_SETTING} to configure a specific implementation.
+ * @param settings settings
+ * @param networkService network service
+ * @param bigArrays big array allocator
+ * @param pageCacheRecycler page cache recycler instance
+ * @param circuitBreakerService circuit breaker service instance
+ * @param threadPool thread pool instance
+ * @param xContentRegistry XContent registry instance
+ * @param dispatcher dispatcher instance
+ * @param clusterSettings cluster settings
+ * @param tracer tracer instance
+ */
+ @Override
+ public Map> getHttpTransports(
+ Settings settings,
+ ThreadPool threadPool,
+ BigArrays bigArrays,
+ PageCacheRecycler pageCacheRecycler,
+ CircuitBreakerService circuitBreakerService,
+ NamedXContentRegistry xContentRegistry,
+ NetworkService networkService,
+ HttpServerTransport.Dispatcher dispatcher,
+ ClusterSettings clusterSettings,
+ Tracer tracer
+ ) {
+ return Collections.singletonMap(
+ REACTOR_NETTY_HTTP_TRANSPORT_NAME,
+ () -> new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry,
+ dispatcher,
+ clusterSettings,
+ getSharedGroupFactory(settings),
+ tracer
+ )
+ );
+ }
+
+ private SharedGroupFactory getSharedGroupFactory(Settings settings) {
+ final SharedGroupFactory groupFactory = this.groupFactory.get();
+ if (groupFactory != null) {
+ assert groupFactory.getSettings().equals(settings) : "Different settings than originally provided";
+ return groupFactory;
+ } else {
+ this.groupFactory.set(new SharedGroupFactory(settings));
+ return this.groupFactory.get();
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/SharedGroupFactory.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/SharedGroupFactory.java
new file mode 100644
index 0000000000000..ab7de33c8e673
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/SharedGroupFactory.java
@@ -0,0 +1,164 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.transport.reactor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.concurrent.AbstractRefCounted;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.http.reactor.netty4.ReactorNetty4HttpServerTransport;
+import org.opensearch.transport.TcpTransport;
+import org.opensearch.transport.reactor.netty4.ReactorNetty4Transport;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.util.concurrent.Future;
+
+import static org.opensearch.common.util.concurrent.OpenSearchExecutors.daemonThreadFactory;
+
+/**
+ * Creates and returns {@link io.netty.channel.EventLoopGroup} instances. It will return a shared group for
+ * both {@link #getHttpGroup()} and {@link #getTransportGroup()} if
+ * {@link org.opensearch.http.reactor.netty4.ReactorNetty4HttpServerTransport#SETTING_HTTP_WORKER_COUNT} is configured to be 0.
+ * If that setting is not 0, then it will return a different group in the {@link #getHttpGroup()} call.
+ */
+public final class SharedGroupFactory {
+
+ private static final Logger logger = LogManager.getLogger(SharedGroupFactory.class);
+
+ private final Settings settings;
+ private final int workerCount;
+ private final int httpWorkerCount;
+
+ private RefCountedGroup genericGroup;
+ private SharedGroup dedicatedHttpGroup;
+
+ /**
+ * Creates new shared group factory instance from settings
+ * @param settings settings
+ */
+ public SharedGroupFactory(Settings settings) {
+ this.settings = settings;
+ this.workerCount = ReactorNetty4Transport.SETTING_WORKER_COUNT.get(settings);
+ this.httpWorkerCount = ReactorNetty4HttpServerTransport.SETTING_HTTP_WORKER_COUNT.get(settings);
+ }
+
+ Settings getSettings() {
+ return settings;
+ }
+
+ /**
+ * Gets the number of configured transport workers
+ * @return the number of configured transport workers
+ */
+ public int getTransportWorkerCount() {
+ return workerCount;
+ }
+
+ /**
+ * Gets transport shared group
+ * @return transport shared group
+ */
+ public synchronized SharedGroup getTransportGroup() {
+ return getGenericGroup();
+ }
+
+ /**
+ * Gets HTTP transport shared group
+ * @return HTTP transport shared group
+ */
+ public synchronized SharedGroup getHttpGroup() {
+ if (httpWorkerCount == 0) {
+ return getGenericGroup();
+ } else {
+ if (dedicatedHttpGroup == null) {
+ NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(
+ httpWorkerCount,
+ daemonThreadFactory(settings, HttpServerTransport.HTTP_SERVER_WORKER_THREAD_NAME_PREFIX)
+ );
+ dedicatedHttpGroup = new SharedGroup(new RefCountedGroup(eventLoopGroup));
+ }
+ return dedicatedHttpGroup;
+ }
+ }
+
+ private SharedGroup getGenericGroup() {
+ if (genericGroup == null) {
+ EventLoopGroup eventLoopGroup = new NioEventLoopGroup(
+ workerCount,
+ daemonThreadFactory(settings, TcpTransport.TRANSPORT_WORKER_THREAD_NAME_PREFIX)
+ );
+ this.genericGroup = new RefCountedGroup(eventLoopGroup);
+ } else {
+ genericGroup.incRef();
+ }
+ return new SharedGroup(genericGroup);
+ }
+
+ private static class RefCountedGroup extends AbstractRefCounted {
+
+ public static final String NAME = "ref-counted-event-loop-group";
+ private final EventLoopGroup eventLoopGroup;
+
+ private RefCountedGroup(EventLoopGroup eventLoopGroup) {
+ super(NAME);
+ this.eventLoopGroup = eventLoopGroup;
+ }
+
+ @Override
+ protected void closeInternal() {
+ Future> shutdownFuture = eventLoopGroup.shutdownGracefully(0, 5, TimeUnit.SECONDS);
+ shutdownFuture.awaitUninterruptibly();
+ if (shutdownFuture.isSuccess() == false) {
+ logger.warn("Error closing netty event loop group", shutdownFuture.cause());
+ }
+ }
+ }
+
+ /**
+ * Wraps the {@link RefCountedGroup}. Calls {@link RefCountedGroup#decRef()} on close. After close,
+ * this wrapped instance can no longer be used.
+ */
+ public static class SharedGroup {
+
+ private final RefCountedGroup refCountedGroup;
+
+ private final AtomicBoolean isOpen = new AtomicBoolean(true);
+
+ private SharedGroup(RefCountedGroup refCountedGroup) {
+ this.refCountedGroup = refCountedGroup;
+ }
+
+ /**
+ * Gets Netty's {@link EventLoopGroup} instance
+ * @return Netty's {@link EventLoopGroup} instance
+ */
+ public EventLoopGroup getLowLevelGroup() {
+ return refCountedGroup.eventLoopGroup;
+ }
+
+ /**
+ * Decreases the reference to underlying {@link EventLoopGroup} instance
+ */
+ public void shutdown() {
+ if (isOpen.compareAndSet(true, false)) {
+ refCountedGroup.decRef();
+ }
+ }
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/Netty4Utils.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/Netty4Utils.java
new file mode 100644
index 0000000000000..8ec432b7dd5cd
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/Netty4Utils.java
@@ -0,0 +1,142 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+package org.opensearch.transport.reactor.netty4;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+import org.opensearch.ExceptionsHelper;
+import org.opensearch.common.Booleans;
+import org.opensearch.common.concurrent.CompletableContext;
+import org.opensearch.core.common.bytes.BytesArray;
+import org.opensearch.core.common.bytes.BytesReference;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.NettyRuntime;
+
+/**
+ * Shameless copy of Netty4Utils from transport-netty4 module
+ */
+public final class Netty4Utils {
+ private static final AtomicBoolean isAvailableProcessorsSet = new AtomicBoolean();
+
+ /**
+ * Utility class
+ */
+ private Netty4Utils() {}
+
+ /**
+ * Set the number of available processors that Netty uses for sizing various resources (e.g., thread pools).
+ *
+ * @param availableProcessors the number of available processors
+ * @throws IllegalStateException if available processors was set previously and the specified value does not match the already-set value
+ */
+ public static void setAvailableProcessors(final int availableProcessors) {
+ // we set this to false in tests to avoid tests that randomly set processors from stepping on each other
+ final boolean set = Booleans.parseBoolean(System.getProperty("opensearch.set.netty.runtime.available.processors", "true"));
+ if (!set) {
+ return;
+ }
+
+ /*
+ * This can be invoked twice, once from Netty4Transport and another time from Netty4HttpServerTransport; however,
+ * Netty4Runtime#availableProcessors forbids settings the number of processors twice so we prevent double invocation here.
+ */
+ if (isAvailableProcessorsSet.compareAndSet(false, true)) {
+ NettyRuntime.setAvailableProcessors(availableProcessors);
+ } else if (availableProcessors != NettyRuntime.availableProcessors()) {
+ /*
+ * We have previously set the available processors yet either we are trying to set it to a different value now or there is a bug
+ * in Netty and our previous value did not take, bail.
+ */
+ final String message = String.format(
+ Locale.ROOT,
+ "available processors value [%d] did not match current value [%d]",
+ availableProcessors,
+ NettyRuntime.availableProcessors()
+ );
+ throw new IllegalStateException(message);
+ }
+ }
+
+ /**
+ * Turns the given BytesReference into a ByteBuf. Note: the returned ByteBuf will reference the internal
+ * pages of the BytesReference. Don't free the bytes of reference before the ByteBuf goes out of scope.
+ * @param reference reference to convert
+ */
+ public static ByteBuf toByteBuf(final BytesReference reference) {
+ if (reference.length() == 0) {
+ return Unpooled.EMPTY_BUFFER;
+ }
+ final BytesRefIterator iterator = reference.iterator();
+ // usually we have one, two, or three components from the header, the message, and a buffer
+ final List buffers = new ArrayList<>(3);
+ try {
+ BytesRef slice;
+ while ((slice = iterator.next()) != null) {
+ buffers.add(Unpooled.wrappedBuffer(slice.bytes, slice.offset, slice.length));
+ }
+
+ if (buffers.size() == 1) {
+ return buffers.get(0);
+ } else {
+ CompositeByteBuf composite = Unpooled.compositeBuffer(buffers.size());
+ composite.addComponents(true, buffers);
+ return composite;
+ }
+ } catch (IOException ex) {
+ throw new AssertionError("no IO happens here", ex);
+ }
+ }
+
+ /**
+ * Wraps the given ChannelBuffer with a BytesReference
+ * @param buffer buffer to convert
+ */
+ public static BytesReference toBytesReference(final ByteBuf buffer) {
+ final int readableBytes = buffer.readableBytes();
+ if (readableBytes == 0) {
+ return BytesArray.EMPTY;
+ } else if (buffer.hasArray()) {
+ return new BytesArray(buffer.array(), buffer.arrayOffset() + buffer.readerIndex(), readableBytes);
+ } else {
+ final ByteBuffer[] byteBuffers = buffer.nioBuffers();
+ return BytesReference.fromByteBuffers(byteBuffers);
+ }
+ }
+
+ /**
+ * Add completion listener to ChannelFuture
+ * @param channelFuture ChannelFuture to add listener to
+ * @param context completion listener context
+ */
+ public static void addListener(ChannelFuture channelFuture, CompletableContext context) {
+ channelFuture.addListener(f -> {
+ if (f.isSuccess()) {
+ context.complete(null);
+ } else {
+ Throwable cause = f.cause();
+ if (cause instanceof Error) {
+ ExceptionsHelper.maybeDieOnAnotherThread(cause);
+ context.completeExceptionally(new Exception(cause));
+ } else {
+ context.completeExceptionally((Exception) cause);
+ }
+ }
+ });
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/ReactorNetty4Transport.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/ReactorNetty4Transport.java
new file mode 100644
index 0000000000000..b3e92f58c540a
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/ReactorNetty4Transport.java
@@ -0,0 +1,35 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.transport.reactor.netty4;
+
+import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Setting.Property;
+import org.opensearch.common.util.concurrent.OpenSearchExecutors;
+
+import reactor.netty.tcp.TcpServer;
+
+/**
+ * The transport implementations based on Reactor Netty (see please {@link TcpServer}).
+ */
+public class ReactorNetty4Transport {
+ /**
+ * The number of Netty workers
+ */
+ public static final Setting SETTING_WORKER_COUNT = new Setting<>(
+ "transport.netty.worker_count",
+ (s) -> Integer.toString(OpenSearchExecutors.allocatedProcessors(s)),
+ (s) -> Setting.parseInt(s, 1, "transport.netty.worker_count"),
+ Property.NodeScope
+ );
+
+ /**
+ * Default constructor
+ */
+ public ReactorNetty4Transport() {}
+}
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/package-info.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/package-info.java
new file mode 100644
index 0000000000000..921bca104c6fe
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/netty4/package-info.java
@@ -0,0 +1,12 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/**
+ * The new transport implementations based on Reactor Netty.
+ */
+package org.opensearch.transport.reactor.netty4;
diff --git a/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/package-info.java b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/package-info.java
new file mode 100644
index 0000000000000..2f36ebb7f11f8
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/java/org/opensearch/transport/reactor/package-info.java
@@ -0,0 +1,12 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/**
+ * The experimental network plugin that introduces new transport implementations based on Reactor Netty.
+ */
+package org.opensearch.transport.reactor;
diff --git a/plugins/transport-reactor-netty4/src/main/plugin-metadata/plugin-security.policy b/plugins/transport-reactor-netty4/src/main/plugin-metadata/plugin-security.policy
new file mode 100644
index 0000000000000..4f2dcde995338
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/main/plugin-metadata/plugin-security.policy
@@ -0,0 +1,24 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+grant codeBase "${codebase.netty-common}" {
+ // for reading the system-wide configuration for the backlog of established sockets
+ permission java.io.FilePermission "/proc/sys/net/core/somaxconn", "read";
+
+ // netty makes and accepts socket connections
+ permission java.net.SocketPermission "*", "accept,connect";
+
+ // Netty sets custom classloader for some of its internal threads
+ permission java.lang.RuntimePermission "*", "setContextClassLoader";
+};
+
+grant codeBase "${codebase.netty-transport}" {
+ // Netty NioEventLoop wants to change this, because of https://bugs.openjdk.java.net/browse/JDK-6427854
+ // the bug says it only happened rarely, and that its fixed, but apparently it still happens rarely!
+ permission java.util.PropertyPermission "sun.nio.ch.bugLevel", "write";
+};
diff --git a/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorHttpClient.java b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorHttpClient.java
new file mode 100644
index 0000000000000..443ecd0f40ead
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorHttpClient.java
@@ -0,0 +1,208 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.common.collect.Tuple;
+import org.opensearch.tasks.Task;
+
+import java.io.Closeable;
+import java.net.InetSocketAddress;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.handler.codec.http.DefaultFullHttpRequest;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.EmptyHttpHeaders;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http2.HttpConversionUtil;
+import io.netty.resolver.DefaultAddressResolverGroup;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.Mono;
+import reactor.core.publisher.ParallelFlux;
+import reactor.netty.http.client.HttpClient;
+
+import static io.netty.handler.codec.http.HttpHeaderNames.HOST;
+import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+/**
+ * Tiny helper to send http requests over netty.
+ */
+class ReactorHttpClient implements Closeable {
+ private final boolean compression;
+
+ static Collection returnHttpResponseBodies(Collection responses) {
+ List list = new ArrayList<>(responses.size());
+ for (FullHttpResponse response : responses) {
+ list.add(response.content().toString(StandardCharsets.UTF_8));
+ }
+ return list;
+ }
+
+ static Collection returnOpaqueIds(Collection responses) {
+ List list = new ArrayList<>(responses.size());
+ for (HttpResponse response : responses) {
+ list.add(response.headers().get(Task.X_OPAQUE_ID));
+ }
+ return list;
+ }
+
+ ReactorHttpClient(boolean compression) {
+ this.compression = compression;
+ }
+
+ static ReactorHttpClient create() {
+ return create(true);
+ }
+
+ static ReactorHttpClient create(boolean compression) {
+ return new ReactorHttpClient(compression);
+ }
+
+ public List get(InetSocketAddress remoteAddress, String... uris) throws InterruptedException {
+ return get(remoteAddress, false, uris);
+ }
+
+ public List get(InetSocketAddress remoteAddress, boolean ordered, String... uris) throws InterruptedException {
+ final List requests = new ArrayList<>(uris.length);
+
+ for (int i = 0; i < uris.length; i++) {
+ final FullHttpRequest httpRequest = new DefaultFullHttpRequest(HTTP_1_1, HttpMethod.GET, uris[i]);
+ httpRequest.headers().add(HOST, "localhost");
+ httpRequest.headers().add("X-Opaque-ID", String.valueOf(i));
+ httpRequest.headers().add(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), "http");
+ requests.add(httpRequest);
+ }
+
+ return sendRequests(remoteAddress, requests, ordered);
+ }
+
+ public final Collection post(InetSocketAddress remoteAddress, List> urisAndBodies)
+ throws InterruptedException {
+ return processRequestsWithBody(HttpMethod.POST, remoteAddress, urisAndBodies);
+ }
+
+ public final FullHttpResponse send(InetSocketAddress remoteAddress, FullHttpRequest httpRequest) throws InterruptedException {
+ final List responses = sendRequests(remoteAddress, Collections.singleton(httpRequest), false);
+ assert responses.size() == 1 : "expected 1 and only 1 http response";
+ return responses.get(0);
+ }
+
+ public final FullHttpResponse send(InetSocketAddress remoteAddress, FullHttpRequest httpRequest, HttpContent content)
+ throws InterruptedException {
+ final List responses = sendRequests(
+ remoteAddress,
+ Collections.singleton(
+ new DefaultFullHttpRequest(
+ httpRequest.protocolVersion(),
+ httpRequest.method(),
+ httpRequest.uri(),
+ content.content(),
+ httpRequest.headers(),
+ httpRequest.trailingHeaders()
+ )
+ ),
+ false
+ );
+ assert responses.size() == 1 : "expected 1 and only 1 http response";
+ return responses.get(0);
+ }
+
+ public final Collection put(InetSocketAddress remoteAddress, List> urisAndBodies)
+ throws InterruptedException {
+ return processRequestsWithBody(HttpMethod.PUT, remoteAddress, urisAndBodies);
+ }
+
+ private List processRequestsWithBody(
+ HttpMethod method,
+ InetSocketAddress remoteAddress,
+ List> urisAndBodies
+ ) throws InterruptedException {
+ List requests = new ArrayList<>(urisAndBodies.size());
+ for (int i = 0; i < urisAndBodies.size(); ++i) {
+ final Tuple uriAndBody = urisAndBodies.get(i);
+ ByteBuf content = Unpooled.copiedBuffer(uriAndBody.v2(), StandardCharsets.UTF_8);
+ FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, method, uriAndBody.v1(), content);
+ request.headers().add(HttpHeaderNames.HOST, "localhost");
+ request.headers().add(HttpHeaderNames.CONTENT_LENGTH, content.readableBytes());
+ request.headers().add(HttpHeaderNames.CONTENT_TYPE, "application/json");
+ request.headers().add(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), "http");
+ request.headers().add("X-Opaque-ID", String.valueOf(i));
+ requests.add(request);
+ }
+ return sendRequests(remoteAddress, requests, false);
+ }
+
+ private List sendRequests(
+ final InetSocketAddress remoteAddress,
+ final Collection requests,
+ boolean orderer
+ ) {
+ final NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(1);
+ try {
+ final HttpClient client = HttpClient.newConnection()
+ .resolver(DefaultAddressResolverGroup.INSTANCE)
+ .runOn(eventLoopGroup)
+ .host(remoteAddress.getHostString())
+ .port(remoteAddress.getPort())
+ .compress(compression);
+
+ @SuppressWarnings("unchecked")
+ final Mono[] monos = requests.stream()
+ .map(
+ request -> client.headers(h -> h.add(request.headers()))
+ .baseUrl(request.getUri())
+ .request(request.method())
+ .send(Mono.fromSupplier(() -> request.content()))
+ .responseSingle(
+ (r, body) -> body.switchIfEmpty(Mono.just(Unpooled.EMPTY_BUFFER))
+ .map(
+ b -> new DefaultFullHttpResponse(
+ r.version(),
+ r.status(),
+ b.retain(),
+ r.responseHeaders(),
+ EmptyHttpHeaders.INSTANCE
+ )
+ )
+ )
+ )
+ .toArray(Mono[]::new);
+
+ if (orderer == false) {
+ return ParallelFlux.from(monos).sequential().collectList().block();
+ } else {
+ return Flux.concat(monos).flatMapSequential(r -> Mono.just(r)).collectList().block();
+ }
+ } finally {
+ eventLoopGroup.shutdownGracefully().awaitUninterruptibly();
+ }
+ }
+
+ @Override
+ public void close() {
+
+ }
+}
diff --git a/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4BadRequestTests.java b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4BadRequestTests.java
new file mode 100644
index 0000000000000..00ca378a4e46b
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4BadRequestTests.java
@@ -0,0 +1,122 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.opensearch.OpenSearchException;
+import org.opensearch.common.network.NetworkService;
+import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.MockBigArrays;
+import org.opensearch.common.util.MockPageCacheRecycler;
+import org.opensearch.common.util.concurrent.ThreadContext;
+import org.opensearch.core.common.transport.TransportAddress;
+import org.opensearch.core.indices.breaker.NoneCircuitBreakerService;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.http.HttpTransportSettings;
+import org.opensearch.rest.BytesRestResponse;
+import org.opensearch.rest.RestChannel;
+import org.opensearch.rest.RestRequest;
+import org.opensearch.telemetry.tracing.noop.NoopTracer;
+import org.opensearch.test.OpenSearchTestCase;
+import org.opensearch.threadpool.TestThreadPool;
+import org.opensearch.threadpool.ThreadPool;
+import org.opensearch.transport.reactor.SharedGroupFactory;
+import org.junit.After;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.util.ReferenceCounted;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+
+public class ReactorNetty4BadRequestTests extends OpenSearchTestCase {
+
+ private NetworkService networkService;
+ private MockBigArrays bigArrays;
+ private ThreadPool threadPool;
+
+ @Before
+ public void setup() throws Exception {
+ networkService = new NetworkService(Collections.emptyList());
+ bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
+ threadPool = new TestThreadPool("test");
+ }
+
+ @After
+ public void shutdown() throws Exception {
+ terminate(threadPool);
+ }
+
+ public void testBadParameterEncoding() throws Exception {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+ @Override
+ public void dispatchRequest(RestRequest request, RestChannel channel, ThreadContext threadContext) {
+ fail();
+ }
+
+ @Override
+ public void dispatchBadRequest(RestChannel channel, ThreadContext threadContext, Throwable cause) {
+ try {
+ final Exception e = cause instanceof Exception ? (Exception) cause : new OpenSearchException(cause);
+ channel.sendResponse(new BytesRestResponse(channel, RestStatus.BAD_REQUEST, e));
+ } catch (final IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+ };
+
+ Settings settings = Settings.builder().put(HttpTransportSettings.SETTING_HTTP_PORT.getKey(), getPortRange()).build();
+ try (
+ HttpServerTransport httpServerTransport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
+ new SharedGroupFactory(Settings.EMPTY),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ httpServerTransport.start();
+ final TransportAddress transportAddress = randomFrom(httpServerTransport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient nettyHttpClient = ReactorHttpClient.create()) {
+ final List responses = nettyHttpClient.get(transportAddress.address(), "/_cluster/settings?pretty=%");
+
+ try {
+ assertThat(responses, hasSize(1));
+ final FullHttpResponse response = responses.get(0);
+ assertThat(response.status().code(), equalTo(400));
+ final Collection responseBodies = ReactorHttpClient.returnHttpResponseBodies(responses);
+ assertThat(responseBodies, hasSize(1));
+ final String body = responseBodies.iterator().next();
+ assertThat(body, containsString("\"type\":\"bad_parameter_exception\""));
+ assertThat(
+ body,
+ containsString("\"reason\":\"java.lang.IllegalArgumentException: partial escape sequence at end of string: %/\"")
+ );
+ } finally {
+ responses.forEach(ReferenceCounted::release);
+ }
+ }
+ }
+ }
+
+}
diff --git a/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransportTests.java b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransportTests.java
new file mode 100644
index 0000000000000..15a5b04c802a4
--- /dev/null
+++ b/plugins/transport-reactor-netty4/src/test/java/org/opensearch/http/reactor/netty4/ReactorNetty4HttpServerTransportTests.java
@@ -0,0 +1,579 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.http.reactor.netty4;
+
+import org.apache.logging.log4j.message.ParameterizedMessage;
+import org.opensearch.common.network.NetworkAddress;
+import org.opensearch.common.network.NetworkService;
+import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.common.util.MockBigArrays;
+import org.opensearch.common.util.MockPageCacheRecycler;
+import org.opensearch.common.util.concurrent.ThreadContext;
+import org.opensearch.core.common.bytes.BytesArray;
+import org.opensearch.core.common.transport.TransportAddress;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.core.indices.breaker.NoneCircuitBreakerService;
+import org.opensearch.http.BindHttpException;
+import org.opensearch.http.CorsHandler;
+import org.opensearch.http.HttpServerTransport;
+import org.opensearch.http.HttpTransportSettings;
+import org.opensearch.http.NullDispatcher;
+import org.opensearch.rest.BytesRestResponse;
+import org.opensearch.rest.RestChannel;
+import org.opensearch.rest.RestRequest;
+import org.opensearch.telemetry.tracing.noop.NoopTracer;
+import org.opensearch.test.OpenSearchTestCase;
+import org.opensearch.test.rest.FakeRestRequest;
+import org.opensearch.threadpool.TestThreadPool;
+import org.opensearch.threadpool.ThreadPool;
+import org.opensearch.transport.NettyAllocator;
+import org.opensearch.transport.reactor.SharedGroupFactory;
+import org.junit.After;
+import org.junit.Before;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufUtil;
+import io.netty.buffer.PoolArenaMetric;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocatorMetric;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.http.DefaultFullHttpRequest;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpUtil;
+import io.netty.handler.codec.http.HttpVersion;
+
+import static org.opensearch.core.rest.RestStatus.OK;
+import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN;
+import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ENABLED;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+
+/**
+ * Tests for the {@link ReactorNetty4HttpServerTransport} class.
+ */
+public class ReactorNetty4HttpServerTransportTests extends OpenSearchTestCase {
+
+ private NetworkService networkService;
+ private ThreadPool threadPool;
+ private MockBigArrays bigArrays;
+ private ClusterSettings clusterSettings;
+
+ @Before
+ public void setup() throws Exception {
+ networkService = new NetworkService(Collections.emptyList());
+ threadPool = new TestThreadPool("test");
+ bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
+ clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+ }
+
+ @After
+ public void shutdown() throws Exception {
+ if (threadPool != null) {
+ threadPool.shutdownNow();
+ }
+ threadPool = null;
+ networkService = null;
+ bigArrays = null;
+ clusterSettings = null;
+ }
+
+ /**
+ * Test that {@link ReactorNetty4HttpServerTransport} supports the "Expect: 100-continue" HTTP header
+ * @throws InterruptedException if the client communication with the server is interrupted
+ */
+ public void testExpectContinueHeader() throws InterruptedException {
+ final Settings settings = createSettings();
+ final int contentLength = randomIntBetween(1, HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH.get(settings).bytesAsInt());
+ runExpectHeaderTest(settings, HttpHeaderValues.CONTINUE.toString(), contentLength, HttpResponseStatus.CONTINUE);
+ }
+
+ /**
+ * Test that {@link ReactorNetty4HttpServerTransport} responds to a
+ * 100-continue expectation with too large a content-length
+ * with a 413 status.
+ * @throws InterruptedException if the client communication with the server is interrupted
+ */
+ public void testExpectContinueHeaderContentLengthTooLong() throws InterruptedException {
+ final String key = HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH.getKey();
+ final int maxContentLength = randomIntBetween(1, 104857600);
+ final Settings settings = createBuilderWithPort().put(key, maxContentLength + "b").build();
+ final int contentLength = randomIntBetween(maxContentLength + 1, Integer.MAX_VALUE);
+ runExpectHeaderTest(settings, HttpHeaderValues.CONTINUE.toString(), contentLength, HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE);
+ }
+
+ /**
+ * Test that {@link ReactorNetty4HttpServerTransport} responds to an unsupported expectation with a 417 status.
+ * @throws InterruptedException if the client communication with the server is interrupted
+ */
+ public void testExpectUnsupportedExpectation() throws InterruptedException {
+ Settings settings = createSettings();
+ runExpectHeaderTest(settings, "chocolate=yummy", 0, HttpResponseStatus.EXPECTATION_FAILED);
+ }
+
+ private void runExpectHeaderTest(
+ final Settings settings,
+ final String expectation,
+ final int contentLength,
+ final HttpResponseStatus expectedStatus
+ ) throws InterruptedException {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+ @Override
+ public void dispatchRequest(RestRequest request, RestChannel channel, ThreadContext threadContext) {
+ channel.sendResponse(new BytesRestResponse(OK, BytesRestResponse.TEXT_CONTENT_TYPE, new BytesArray("done")));
+ }
+
+ @Override
+ public void dispatchBadRequest(RestChannel channel, ThreadContext threadContext, Throwable cause) {
+ logger.error(
+ new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())),
+ cause
+ );
+ throw new AssertionError();
+ }
+ };
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ clusterSettings,
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, "/");
+ request.headers().set(HttpHeaderNames.EXPECT, expectation);
+ HttpUtil.setContentLength(request, contentLength);
+
+ // Reactor Netty 4 does not expose 100 CONTINUE response but instead just asks for content
+ final HttpContent continuationRequest = new DefaultHttpContent(Unpooled.EMPTY_BUFFER);
+ final FullHttpResponse continuationResponse = client.send(remoteAddress.address(), request, continuationRequest);
+ try {
+ assertThat(continuationResponse.status(), is(HttpResponseStatus.OK));
+ assertThat(new String(ByteBufUtil.getBytes(continuationResponse.content()), StandardCharsets.UTF_8), is("done"));
+ } finally {
+ continuationResponse.release();
+ }
+ }
+ }
+ }
+
+ public void testBindUnavailableAddress() {
+ Settings initialSettings = createSettings();
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ initialSettings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ new NullDispatcher(),
+ clusterSettings,
+ new SharedGroupFactory(Settings.EMPTY),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+ Settings settings = Settings.builder()
+ .put("http.port", remoteAddress.getPort())
+ .put("network.host", remoteAddress.getAddress())
+ .build();
+ try (
+ ReactorNetty4HttpServerTransport otherTransport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ new NullDispatcher(),
+ clusterSettings,
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ BindHttpException bindHttpException = expectThrows(BindHttpException.class, otherTransport::start);
+ assertEquals("Failed to bind to " + NetworkAddress.format(remoteAddress.address()), bindHttpException.getMessage());
+ }
+ }
+ }
+
+ public void testBadRequest() throws InterruptedException {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+ @Override
+ public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) {
+ logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request));
+ throw new AssertionError();
+ }
+
+ @Override
+ public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) {
+ logger.error("--> Unexpected bad request request");
+ throw new AssertionError(cause);
+ }
+ };
+
+ final Settings settings;
+ final int maxInitialLineLength;
+ final Setting httpMaxInitialLineLengthSetting = HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH;
+ if (randomBoolean()) {
+ maxInitialLineLength = httpMaxInitialLineLengthSetting.getDefault(Settings.EMPTY).bytesAsInt();
+ settings = createSettings();
+ } else {
+ maxInitialLineLength = randomIntBetween(1, 8192);
+ settings = createBuilderWithPort().put(httpMaxInitialLineLengthSetting.getKey(), maxInitialLineLength + "b").build();
+ }
+
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ clusterSettings,
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ final String url = "/" + randomAlphaOfLength(maxInitialLineLength);
+ final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, url);
+
+ final FullHttpResponse response = client.send(remoteAddress.address(), request);
+ try {
+ assertThat(response.status(), equalTo(HttpResponseStatus.REQUEST_URI_TOO_LONG));
+ assertThat(response.content().array().length, equalTo(0));
+ } finally {
+ response.release();
+ }
+ }
+ }
+ }
+
+ public void testDispatchFailed() throws InterruptedException {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+ @Override
+ public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) {
+ throw new RuntimeException("Bad things happen");
+ }
+
+ @Override
+ public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) {
+ logger.error("--> Unexpected bad request request");
+ throw new AssertionError(cause);
+ }
+ };
+
+ final Settings settings = createSettings();
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ clusterSettings,
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/");
+
+ final FullHttpResponse response = client.send(remoteAddress.address(), request);
+ try {
+ assertThat(response.status(), equalTo(HttpResponseStatus.INTERNAL_SERVER_ERROR));
+ assertThat(response.content().array().length, equalTo(0));
+ } finally {
+ response.release();
+ }
+ }
+ }
+ }
+
+ public void testLargeCompressedResponse() throws InterruptedException {
+ final String responseString = randomAlphaOfLength(4 * 1024 * 1024);
+ final String url = "/thing/";
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+
+ @Override
+ public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) {
+ if (url.equals(request.uri())) {
+ channel.sendResponse(new BytesRestResponse(OK, responseString));
+ } else {
+ logger.error("--> Unexpected successful uri [{}]", request.uri());
+ throw new AssertionError();
+ }
+ }
+
+ @Override
+ public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) {
+ logger.error(
+ new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())),
+ cause
+ );
+ throw new AssertionError();
+ }
+
+ };
+
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ Settings.EMPTY,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ clusterSettings,
+ new SharedGroupFactory(Settings.EMPTY),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ DefaultFullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, url);
+ request.headers().add(HttpHeaderNames.ACCEPT_ENCODING, randomFrom("deflate", "gzip"));
+ long numOfHugeAllocations = getHugeAllocationCount();
+ final FullHttpResponse response = client.send(remoteAddress.address(), request);
+ try {
+ assertThat(getHugeAllocationCount(), equalTo(numOfHugeAllocations));
+ assertThat(response.status(), equalTo(HttpResponseStatus.OK));
+ byte[] bytes = new byte[response.content().readableBytes()];
+ response.content().readBytes(bytes);
+ assertThat(new String(bytes, StandardCharsets.UTF_8), equalTo(responseString));
+ } finally {
+ response.release();
+ }
+ }
+ }
+ }
+
+ private long getHugeAllocationCount() {
+ long numOfHugAllocations = 0;
+ ByteBufAllocator allocator = NettyAllocator.getAllocator();
+ assert allocator instanceof NettyAllocator.NoDirectBuffers;
+ ByteBufAllocator delegate = ((NettyAllocator.NoDirectBuffers) allocator).getDelegate();
+ if (delegate instanceof PooledByteBufAllocator) {
+ PooledByteBufAllocatorMetric metric = ((PooledByteBufAllocator) delegate).metric();
+ numOfHugAllocations = metric.heapArenas().stream().mapToLong(PoolArenaMetric::numHugeAllocations).sum();
+ }
+ return numOfHugAllocations;
+ }
+
+ public void testCorsRequest() throws InterruptedException {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+
+ @Override
+ public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) {
+ logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request));
+ throw new AssertionError();
+ }
+
+ @Override
+ public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) {
+ logger.error(
+ new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())),
+ cause
+ );
+ throw new AssertionError();
+ }
+
+ };
+
+ final Settings settings = createBuilderWithPort().put(SETTING_CORS_ENABLED.getKey(), true)
+ .put(SETTING_CORS_ALLOW_ORIGIN.getKey(), "test-cors.org")
+ .build();
+
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+
+ // Test pre-flight request
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.OPTIONS, "/");
+ request.headers().add(CorsHandler.ORIGIN, "test-cors.org");
+ request.headers().add(CorsHandler.ACCESS_CONTROL_REQUEST_METHOD, "POST");
+
+ final FullHttpResponse response = client.send(remoteAddress.address(), request);
+ try {
+ assertThat(response.status(), equalTo(HttpResponseStatus.OK));
+ assertThat(response.headers().get(CorsHandler.ACCESS_CONTROL_ALLOW_ORIGIN), equalTo("test-cors.org"));
+ assertThat(response.headers().get(CorsHandler.VARY), equalTo(CorsHandler.ORIGIN));
+ assertTrue(response.headers().contains(CorsHandler.DATE));
+ } finally {
+ response.release();
+ }
+ }
+
+ // Test short-circuited request
+ try (ReactorHttpClient client = ReactorHttpClient.create()) {
+ final FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/");
+ request.headers().add(CorsHandler.ORIGIN, "google.com");
+
+ final FullHttpResponse response = client.send(remoteAddress.address(), request);
+ try {
+ assertThat(response.status(), equalTo(HttpResponseStatus.FORBIDDEN));
+ } finally {
+ response.release();
+ }
+ }
+ }
+ }
+
+ public void testConnectTimeout() throws Exception {
+ final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() {
+
+ @Override
+ public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) {
+ logger.error("--> Unexpected successful request [{}]", FakeRestRequest.requestToString(request));
+ throw new AssertionError("Should not have received a dispatched request");
+ }
+
+ @Override
+ public void dispatchBadRequest(final RestChannel channel, final ThreadContext threadContext, final Throwable cause) {
+ logger.error(
+ new ParameterizedMessage("--> Unexpected bad request [{}]", FakeRestRequest.requestToString(channel.request())),
+ cause
+ );
+ throw new AssertionError("Should not have received a dispatched request");
+ }
+
+ };
+
+ Settings settings = createBuilderWithPort().put(
+ HttpTransportSettings.SETTING_HTTP_CONNECT_TIMEOUT.getKey(),
+ new TimeValue(randomIntBetween(100, 300))
+ ).build();
+
+ NioEventLoopGroup group = new NioEventLoopGroup();
+ try (
+ ReactorNetty4HttpServerTransport transport = new ReactorNetty4HttpServerTransport(
+ settings,
+ networkService,
+ bigArrays,
+ threadPool,
+ xContentRegistry(),
+ dispatcher,
+ new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS),
+ new SharedGroupFactory(settings),
+ NoopTracer.INSTANCE
+ )
+ ) {
+ transport.start();
+ final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses());
+
+ final CountDownLatch channelClosedLatch = new CountDownLatch(1);
+
+ final Bootstrap clientBootstrap = new Bootstrap().option(ChannelOption.ALLOCATOR, NettyAllocator.getAllocator())
+ .channel(NioSocketChannel.class)
+ .handler(new ChannelInitializer() {
+
+ @Override
+ protected void initChannel(SocketChannel ch) {
+ ch.pipeline().addLast(new ChannelHandlerAdapter() {
+ });
+
+ }
+ })
+ .group(group);
+ ChannelFuture connect = clientBootstrap.connect(remoteAddress.address());
+ connect.channel().closeFuture().addListener(future -> channelClosedLatch.countDown());
+
+ assertTrue("Channel should be closed due to read timeout", channelClosedLatch.await(1, TimeUnit.MINUTES));
+
+ } finally {
+ group.shutdownGracefully().await();
+ }
+ }
+
+ private Settings createSettings() {
+ return createBuilderWithPort().build();
+ }
+
+ private Settings.Builder createBuilderWithPort() {
+ return Settings.builder().put(HttpTransportSettings.SETTING_HTTP_PORT.getKey(), getPortRange());
+ }
+}
diff --git a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java
index aabc3aee8887f..2675e9b62de35 100644
--- a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java
+++ b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java
@@ -78,7 +78,7 @@
* PercolatorFieldMapper#createQueryBuilderField(...) method). Using the query builders writable contract. This test
* does best effort verifying that we don't break bwc for query builders between the first previous major version and
* the latest current major release.
- *
+ *
* The queries to test are specified in json format, which turns out to work because we tend break here rarely. If the
* json format of a query being tested here then feel free to change this.
*/
diff --git a/qa/no-bootstrap-tests/src/test/java/org/opensearch/bootstrap/SpawnerNoBootstrapTests.java b/qa/no-bootstrap-tests/src/test/java/org/opensearch/bootstrap/SpawnerNoBootstrapTests.java
index c3c332aecfd4c..8ca90791f649e 100644
--- a/qa/no-bootstrap-tests/src/test/java/org/opensearch/bootstrap/SpawnerNoBootstrapTests.java
+++ b/qa/no-bootstrap-tests/src/test/java/org/opensearch/bootstrap/SpawnerNoBootstrapTests.java
@@ -65,7 +65,7 @@
/**
* Create a simple "daemon controller", put it in the right place and check that it runs.
- *
+ *
* Extends LuceneTestCase rather than OpenSearchTestCase as OpenSearchTestCase installs a system call filter, and
* that prevents the Spawner class from doing its job. Also needs to run in a separate JVM to other
* tests that extend OpenSearchTestCase for the same reason.
diff --git a/qa/os/build.gradle b/qa/os/build.gradle
index 66c6525439dac..082ed5277575a 100644
--- a/qa/os/build.gradle
+++ b/qa/os/build.gradle
@@ -70,6 +70,11 @@ tasks.dependenciesInfo.enabled = false
tasks.thirdPartyAudit.ignoreMissingClasses()
+tasks.thirdPartyAudit.ignoreViolations(
+ 'org.apache.logging.log4j.core.util.internal.UnsafeUtil',
+ 'org.apache.logging.log4j.core.util.internal.UnsafeUtil$1'
+)
+
tasks.register('destructivePackagingTest') {
dependsOn 'destructiveDistroTest'
}
diff --git a/qa/os/src/test/java/org/opensearch/packaging/test/PackagingTestCase.java b/qa/os/src/test/java/org/opensearch/packaging/test/PackagingTestCase.java
index 02a613be320c2..4bb3877fc04a8 100644
--- a/qa/os/src/test/java/org/opensearch/packaging/test/PackagingTestCase.java
+++ b/qa/os/src/test/java/org/opensearch/packaging/test/PackagingTestCase.java
@@ -441,7 +441,7 @@ public static Path createTempDir(String prefix) throws IOException {
/**
* Run the given action with a temporary copy of the config directory.
- *
+ *
* Files under the path passed to the action may be modified as necessary for the
* test to execute, and running OpenSearch with {@link #startOpenSearch()} will
* use the temporary directory.
diff --git a/qa/os/src/test/java/org/opensearch/packaging/util/FileMatcher.java b/qa/os/src/test/java/org/opensearch/packaging/util/FileMatcher.java
index 7904d1a046916..958de24848178 100644
--- a/qa/os/src/test/java/org/opensearch/packaging/util/FileMatcher.java
+++ b/qa/os/src/test/java/org/opensearch/packaging/util/FileMatcher.java
@@ -51,7 +51,7 @@
/**
* Asserts that a file at a path matches its status as Directory/File, and its owner. If on a posix system, also matches the permission
* set is what we expect.
- *
+ *
* This class saves information about its failed matches in instance variables and so instances should not be reused
*/
public class FileMatcher extends TypeSafeMatcher {
diff --git a/qa/os/src/test/java/org/opensearch/packaging/util/Installation.java b/qa/os/src/test/java/org/opensearch/packaging/util/Installation.java
index 25cefa948ff10..26af39d66cad3 100644
--- a/qa/os/src/test/java/org/opensearch/packaging/util/Installation.java
+++ b/qa/os/src/test/java/org/opensearch/packaging/util/Installation.java
@@ -137,7 +137,7 @@ public static Installation ofContainer(Shell sh, Distribution distribution) {
/**
* Returns the user that owns this installation.
- *
+ *
* For packages this is root, and for archives it is the user doing the installation.
*/
public String getOwner() {
diff --git a/qa/os/src/test/java/org/opensearch/packaging/util/Packages.java b/qa/os/src/test/java/org/opensearch/packaging/util/Packages.java
index b80ae422bda9a..e9ebf28042b46 100644
--- a/qa/os/src/test/java/org/opensearch/packaging/util/Packages.java
+++ b/qa/os/src/test/java/org/opensearch/packaging/util/Packages.java
@@ -194,11 +194,11 @@ private static void verifyInstallation(Installation opensearch, Distribution dis
// we shell out here because java's posix file permission view doesn't support special modes
assertThat(opensearch.config, file(Directory, "root", "opensearch", p750));
- assertThat(sh.run("find \"" + opensearch.config + "\" -maxdepth 0 -printf \"%m\"").stdout, containsString("2750"));
+ assertThat(sh.run("find \"" + opensearch.config + "\" -maxdepth 0 -printf \"%m\"").stdout, containsString("750"));
final Path jvmOptionsDirectory = opensearch.config.resolve("jvm.options.d");
assertThat(jvmOptionsDirectory, file(Directory, "root", "opensearch", p750));
- assertThat(sh.run("find \"" + jvmOptionsDirectory + "\" -maxdepth 0 -printf \"%m\"").stdout, containsString("2750"));
+ assertThat(sh.run("find \"" + jvmOptionsDirectory + "\" -maxdepth 0 -printf \"%m\"").stdout, containsString("750"));
Stream.of("opensearch.keystore", "opensearch.yml", "jvm.options", "log4j2.properties")
.forEach(configFile -> assertThat(opensearch.config(configFile), file(File, "root", "opensearch", p660)));
diff --git a/qa/remote-clusters/src/test/java/org/opensearch/cluster/remote/test/RemoteClustersIT.java b/qa/remote-clusters/src/test/java/org/opensearch/cluster/remote/test/RemoteClustersIT.java
index dbea8db1a12fa..c38fcc468c673 100644
--- a/qa/remote-clusters/src/test/java/org/opensearch/cluster/remote/test/RemoteClustersIT.java
+++ b/qa/remote-clusters/src/test/java/org/opensearch/cluster/remote/test/RemoteClustersIT.java
@@ -42,11 +42,13 @@
import org.opensearch.client.cluster.RemoteInfoRequest;
import org.opensearch.client.indices.CreateIndexRequest;
import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.xcontent.XContentFactory;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
+import java.util.concurrent.TimeUnit;
public class RemoteClustersIT extends AbstractMultiClusterRemoteTestCase {
@@ -112,7 +114,7 @@ public void testSniffModeConnectionFails() throws IOException {
assertFalse(rci.isConnected());
}
- public void testHAProxyModeConnectionWorks() throws IOException {
+ public void testHAProxyModeConnectionWorks() throws Exception {
String proxyAddress = "haproxy:9600";
logger.info("Configuring remote cluster [{}]", proxyAddress);
ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest().persistentSettings(Settings.builder()
@@ -121,12 +123,14 @@ public void testHAProxyModeConnectionWorks() throws IOException {
.build());
assertTrue(cluster1Client().cluster().putSettings(request, RequestOptions.DEFAULT).isAcknowledged());
- RemoteConnectionInfo rci = cluster1Client().cluster().remoteInfo(new RemoteInfoRequest(), RequestOptions.DEFAULT).getInfos().get(0);
- logger.info("Connection info: {}", rci);
- if (!rci.isConnected()) {
- logger.info("Cluster health: {}", cluster1Client().cluster().health(new ClusterHealthRequest(), RequestOptions.DEFAULT));
- }
- assertTrue(rci.isConnected());
+ assertBusy(() -> {
+ RemoteConnectionInfo rci = cluster1Client().cluster().remoteInfo(new RemoteInfoRequest(), RequestOptions.DEFAULT).getInfos().get(0);
+ logger.info("Connection info: {}", rci);
+ if (!rci.isConnected()) {
+ logger.info("Cluster health: {}", cluster1Client().cluster().health(new ClusterHealthRequest(), RequestOptions.DEFAULT));
+ }
+ assertTrue(rci.isConnected());
+ }, 10, TimeUnit.SECONDS);
assertEquals(2L, cluster1Client().search(
new SearchRequest("haproxynosn:test2"), RequestOptions.DEFAULT).getHits().getTotalHits().value);
diff --git a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java
index aef363058b394..f963f8d221bb5 100644
--- a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java
+++ b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java
@@ -98,11 +98,11 @@ private void waitForSearchableDocs(String index, int shardCount, int replicaCoun
// Verify segment store
assertBusy(() -> {
- /**
- * Use default tabular output and sort response based on shard,segment,primaryOrReplica columns to allow line by
- * line parsing where records related to a segment (e.g. _0) are chunked together with first record belonging
- * to primary while remaining *replicaCount* records belongs to replica copies
- * */
+ /*
+ Use default tabular output and sort response based on shard,segment,primaryOrReplica columns to allow line by
+ line parsing where records related to a segment (e.g. _0) are chunked together with first record belonging
+ to primary while remaining *replicaCount* records belongs to replica copies
+ */
Request segrepStatsRequest = new Request("GET", "/_cat/segments/" + index + "?s=shard,segment,primaryOrReplica");
segrepStatsRequest.addParameter("h", "index,shard,primaryOrReplica,segment,docs.count");
Response segrepStatsResponse = client().performRequest(segrepStatsRequest);
@@ -259,7 +259,8 @@ public void testIndexing() throws Exception {
* This test verifies that during rolling upgrades the segment replication does not break when replica shards can
* be running on older codec versions.
*
- * @throws Exception exception
+ * @throws Exception if index creation fail
+ * @throws UnsupportedOperationException if cluster type is unknown
*/
@AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/7679")
public void testIndexingWithSegRep() throws Exception {
diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/30_geoshape.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/30_geoshape.yml
new file mode 100644
index 0000000000000..e669016cad98a
--- /dev/null
+++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/30_geoshape.yml
@@ -0,0 +1,16 @@
+---
+"Insert Document with geoshape field":
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - '{"index": {"_index": "geo_shape_index_old", "_id":191}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [101.0, 0.0] ]}}'
+ - '{"index": {"_index": "geo_shape_index_old", "_id":219}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [106.0, 0.0] ]}}'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: geo_shape_index_old
+ - match: { hits.total: 2 }
diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_geoshape.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_geoshape.yml
new file mode 100644
index 0000000000000..30a39447905c0
--- /dev/null
+++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_geoshape.yml
@@ -0,0 +1,28 @@
+---
+"Create index with Geoshape field":
+ - do:
+ indices.create:
+ index: geo_shape_index_old
+ body:
+ settings:
+ index:
+ number_of_replicas: 2
+ mappings:
+ "properties":
+ "location":
+ "type": "geo_shape"
+
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - '{"index": {"_index": "geo_shape_index_old", "_id":191}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [101.0, 0.0] ]}}'
+ - '{"index": {"_index": "geo_shape_index_old", "_id":219}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [106.0, 0.0] ]}}'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: geo_shape_index_old
+ - match: { hits.total: 2 }
diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/30_geoshape.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/30_geoshape.yml
new file mode 100644
index 0000000000000..4c7b12a7f1909
--- /dev/null
+++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/30_geoshape.yml
@@ -0,0 +1,61 @@
+---
+"Validate we are able to index documents after upgrade":
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - '{"index": {"_index": "geo_shape_index_old", "_id":191}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [101.0, 0.0] ]}}'
+ - '{"index": {"_index": "geo_shape_index_old", "_id":219}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [106.0, 0.0] ]}}'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: geo_shape_index_old
+ - match: { hits.total: 2 }
+
+
+---
+"Create index with Geoshape field in new cluster":
+ - do:
+ indices.create:
+ index: geo_shape_index_new
+ body:
+ settings:
+ index:
+ number_of_replicas: 2
+ mappings:
+ "properties":
+ "location":
+ "type": "geo_shape"
+
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - '{"index": {"_index": "geo_shape_index_new", "_id":191}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [101.0, 0.0] ]}}'
+ - '{"index": {"_index": "geo_shape_index_new", "_id":219}}'
+ - '{"name": "NEMO Science Museum","location": {"type": "envelope","coordinates": [ [100.0, 1.0], [106.0, 0.0] ]}}'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: geo_shape_index_new
+ - match: { hits.total: 2 }
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: geo_shape_index_new
+ body:
+ aggregations:
+ myaggregation:
+ geo_bounds:
+ field: "location"
+ - match: { hits.total: 2 }
+ - match: { aggregations.myaggregation.bounds.top_left.lat: 0.9999999823048711 }
+ - match: { aggregations.myaggregation.bounds.top_left.lon: 99.99999999068677 }
+ - match: { aggregations.myaggregation.bounds.bottom_right.lat: 0.0 }
+ - match: { aggregations.myaggregation.bounds.bottom_right.lon: 105.99999996833503 }
diff --git a/release-notes/opensearch.release-notes-2.11.0.md b/release-notes/opensearch.release-notes-2.11.0.md
new file mode 100644
index 0000000000000..040cc053469ed
--- /dev/null
+++ b/release-notes/opensearch.release-notes-2.11.0.md
@@ -0,0 +1,68 @@
+## 2023-10-12 Version 2.11.0 Release Notes
+
+## [2.11]
+
+### Added
+- Add coordinator level stats for search latency ([#8386](https://github.com/opensearch-project/OpenSearch/issues/8386))
+- Add metrics for thread_pool task wait time ([#9681](https://github.com/opensearch-project/OpenSearch/pull/9681))
+- Add parallel file download support for remote store based replication ([#8596](https://github.com/opensearch-project/OpenSearch/pull/8596))
+- Async blob read support for S3 plugin ([#9694](https://github.com/opensearch-project/OpenSearch/pull/9694))
+- [Telemetry-Otel] Added support for OtlpGrpcSpanExporter exporter ([#9666](https://github.com/opensearch-project/OpenSearch/pull/9666))
+- Async blob read support for encrypted containers ([#10131](https://github.com/opensearch-project/OpenSearch/pull/10131))
+- Implement Visitor Design pattern in QueryBuilder to enable the capability to traverse through the complex QueryBuilder tree. ([#10110](https://github.com/opensearch-project/OpenSearch/pull/10110))
+- Add capability to restrict async durability mode for remote indexes ([#10189](https://github.com/opensearch-project/OpenSearch/pull/10189))
+- Add Doc Status Counter for Indexing Engine ([#4562](https://github.com/opensearch-project/OpenSearch/issues/4562))
+- Add unreferenced file cleanup count to merge stats ([#10204](https://github.com/opensearch-project/OpenSearch/pull/10204))
+- Configurable merge policy for index with an option to choose from LogByteSize and Tiered merge policy ([#9992](https://github.com/opensearch-project/OpenSearch/pull/9992))
+- [Remote Store] Add support to restrict creation & deletion if system repository and mutation of immutable settings of system repository ([#9839](https://github.com/opensearch-project/OpenSearch/pull/9839))
+- Improve compressed request handling ([#10261](https://github.com/opensearch-project/OpenSearch/pull/10261))
+
+### Dependencies
+- Bump JNA version from 5.5 to 5.13 ([#9963](https://github.com/opensearch-project/OpenSearch/pull/9963))
+- Bump `peter-evans/create-or-update-comment` from 2 to 3 ([#9575](https://github.com/opensearch-project/OpenSearch/pull/9575))
+- Bump `actions/checkout` from 2 to 4 ([#9968](https://github.com/opensearch-project/OpenSearch/pull/9968))
+- Bump OpenTelemetry from 1.26.0 to 1.30.1 ([#9950](https://github.com/opensearch-project/OpenSearch/pull/9950))
+- Bump `org.apache.commons:commons-compress` from 1.23.0 to 1.24.0 ([#9973, #9972](https://github.com/opensearch-project/OpenSearch/pull/9973, https://github.com/opensearch-project/OpenSearch/pull/9972))
+- Bump `com.google.cloud:google-cloud-core-http` from 2.21.1 to 2.23.0 ([#9971](https://github.com/opensearch-project/OpenSearch/pull/9971))
+- Bump `mockito` from 5.4.0 to 5.5.0 ([#10022](https://github.com/opensearch-project/OpenSearch/pull/10022))
+- Bump `bytebuddy` from 1.14.3 to 1.14.7 ([#10022](https://github.com/opensearch-project/OpenSearch/pull/10022))
+- Bump `com.zaxxer:SparseBitSet` from 1.2 to 1.3 ([#10098](https://github.com/opensearch-project/OpenSearch/pull/10098))
+- Bump `tibdex/github-app-token` from 1.5.0 to 2.1.0 ([#10125](https://github.com/opensearch-project/OpenSearch/pull/10125))
+- Bump `org.wiremock:wiremock-standalone` from 2.35.0 to 3.1.0 ([#9752](https://github.com/opensearch-project/OpenSearch/pull/9752))
+- Bump `org.eclipse.jgit` from 6.5.0 to 6.7.0 ([#10147](https://github.com/opensearch-project/OpenSearch/pull/10147))
+- Bump `codecov/codecov-action` from 2 to 3 ([#10209](https://github.com/opensearch-project/OpenSearch/pull/10209))
+- Bump `com.google.http-client:google-http-client-jackson2` from 1.43.2 to 1.43.3 ([#10126](https://github.com/opensearch-project/OpenSearch/pull/10126))
+- Bump `org.xerial.snappy:snappy-java` from 1.1.10.3 to 1.1.10.5 ([#10206](https://github.com/opensearch-project/OpenSearch/pull/10206), [#10299](https://github.com/opensearch-project/OpenSearch/pull/10299))
+- Bump `org.bouncycastle:bcpkix-jdk15to18` from 1.75 to 1.76 ([10219](https://github.com/opensearch-project/OpenSearch/pull/10219))`
+- Bump `org.bouncycastle:bcprov-jdk15to18` from 1.75 to 1.76 ([10219](https://github.com/opensearch-project/OpenSearch/pull/10219))`
+- Bump `org.bouncycastle:bcmail-jdk15to18` from 1.75 to 1.76 ([10219](https://github.com/opensearch-project/OpenSearch/pull/10219))`
+- Bump asm from 9.5 to 9.6 ([#10302](https://github.com/opensearch-project/OpenSearch/pull/10302))
+- Bump netty from 4.1.97.Final to 4.1.99.Final ([#10303](https://github.com/opensearch-project/OpenSearch/pull/10303))
+- Bump `peter-evans/create-pull-request` from 3 to 5 ([#10301](https://github.com/opensearch-project/OpenSearch/pull/10301))
+- Bump `org.apache.avro:avro` from 1.11.2 to 1.11.3 ([#10210](https://github.com/opensearch-project/OpenSearch/pull/10210))
+- Bump `netty` from 4.1.99.Final to 4.1.100.Final ([#10564](https://github.com/opensearch-project/OpenSearch/pull/10564))
+
+### Changed
+- Add instrumentation in rest and network layer. ([#9415](https://github.com/opensearch-project/OpenSearch/pull/9415))
+- Allow parameterization of tests with OpenSearchIntegTestCase.SuiteScopeTestCase annotation ([#9916](https://github.com/opensearch-project/OpenSearch/pull/9916))
+- Add instrumentation in transport service. ([#10042](https://github.com/opensearch-project/OpenSearch/pull/10042))
+- [Tracing Framework] Add support for SpanKind. ([#10122](https://github.com/opensearch-project/OpenSearch/pull/10122))
+- Pass parent filter to inner query in nested query ([#10246](https://github.com/opensearch-project/OpenSearch/pull/10246))
+- Disable concurrent segment search when terminate_after is used ([#10200](https://github.com/opensearch-project/OpenSearch/pull/10200))
+- Add instrumentation in Inbound Handler. ([#100143](https://github.com/opensearch-project/OpenSearch/pull/10143))
+- Enable remote segment upload backpressure by default ([#10356](https://github.com/opensearch-project/OpenSearch/pull/10356))
+- [Remote Store] Add support to reload repository metadata inplace ([#9569](https://github.com/opensearch-project/OpenSearch/pull/9569))
+- [Metrics Framework] Add Metrics framework. ([#10241](https://github.com/opensearch-project/OpenSearch/pull/10241))
+- Updating the separator for RemoteStoreLockManager since underscore is allowed in base64UUID url charset ([#10379](https://github.com/opensearch-project/OpenSearch/pull/10379))
+
+### Removed
+- Remove spurious SGID bit on directories ([#9447](https://github.com/opensearch-project/OpenSearch/pull/9447))
+
+### Fixed
+- Fix ignore_missing parameter has no effect when using template snippet in rename ingest processor ([#9725](https://github.com/opensearch-project/OpenSearch/pull/9725))
+- Fix broken backward compatibility from 2.7 for IndexSorted field indices ([#10045](https://github.com/opensearch-project/OpenSearch/pull/10045))
+- Fix concurrent search NPE when track_total_hits, terminate_after and size=0 are used ([#10082](https://github.com/opensearch-project/OpenSearch/pull/10082))
+- Fix remove ingest processor handing ignore_missing parameter not correctly ([10089](https://github.com/opensearch-project/OpenSearch/pull/10089))
+- Fix registration and initialization of multiple extensions ([10256](https://github.com/opensearch-project/OpenSearch/pull/10256))
+- Fix circular dependency in Settings initialization ([10194](https://github.com/opensearch-project/OpenSearch/pull/10194))
+- Fix Segment Replication ShardLockObtainFailedException bug during index corruption ([10370](https://github.com/opensearch-project/OpenSearch/pull/10370))
diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.sort/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.sort/10_basic.yml
index b9089689b0cf1..3b7ea15164e9f 100644
--- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.sort/10_basic.yml
+++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.sort/10_basic.yml
@@ -156,3 +156,23 @@
query: {"range": { "rank": { "from": 0 } } }
track_total_hits: false
size: 3
+
+---
+"Index Sort half float":
+ - do:
+ catch: bad_request
+ indices.create:
+ index: test
+ body:
+ settings:
+ number_of_shards: 1
+ number_of_replicas: 0
+ index.sort.field: rank
+ mappings:
+ properties:
+ rank:
+ type: half_float
+
+ # This should failed with 400 as half_float is not supported for index sort
+ - match: { status: 400 }
+ - match: { error.type: illegal_argument_exception }
diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/260_sort_mixed.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/260_sort_mixed.yml
index ba2b18eb3b6d0..a04dc308b2a06 100644
--- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/260_sort_mixed.yml
+++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/260_sort_mixed.yml
@@ -20,6 +20,7 @@
properties:
counter:
type: double
+
- do:
bulk:
refresh: true
@@ -119,3 +120,87 @@
- match: { status: 400 }
- match: { error.type: search_phase_execution_exception }
- match: { error.caused_by.reason: "Can't do sort across indices, as a field has [unsigned_long] type in one index, and different type in another index!" }
+
+---
+"search across indices with mixed long and double and float numeric types":
+ - skip:
+ version: " - 2.10.99"
+ reason: half float was broken before 2.11
+
+ - do:
+ indices.create:
+ index: test_1
+ body:
+ mappings:
+ properties:
+ counter:
+ type: long
+
+ - do:
+ indices.create:
+ index: test_2
+ body:
+ mappings:
+ properties:
+ counter:
+ type: double
+
+ - do:
+ indices.create:
+ index: test_3
+ body:
+ mappings:
+ properties:
+ counter:
+ type: half_float
+
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - index:
+ _index: test_1
+ - counter: 223372036854775800
+ - index:
+ _index: test_2
+ - counter: 1223372036854775800.23
+ - index:
+ _index: test_2
+ - counter: 184.4
+ - index:
+ _index: test_3
+ - counter: 187.4
+ - index:
+ _index: test_3
+ - counter: 194.4
+
+ - do:
+ search:
+ index: test_*
+ rest_total_hits_as_int: true
+ body:
+ sort: [{ counter: desc }]
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 5 }
+ - match: { hits.hits.0._index: test_2 }
+ - match: { hits.hits.0._source.counter: 1223372036854775800.23 }
+ - match: { hits.hits.0.sort.0: 1223372036854775800.23 }
+ - match: { hits.hits.1._index: test_1 }
+ - match: { hits.hits.1._source.counter: 223372036854775800 }
+ - match: { hits.hits.1.sort.0: 223372036854775800 }
+ - match: { hits.hits.2._index: test_3 }
+ - match: { hits.hits.2._source.counter: 194.4 }
+
+ - do:
+ search:
+ index: test_*
+ rest_total_hits_as_int: true
+ body:
+ sort: [{ counter: asc }]
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 5 }
+ - match: { hits.hits.0._index: test_2 }
+ - match: { hits.hits.0._source.counter: 184.4 }
+ - match: { hits.hits.0.sort.0: 184.4 }
+ - match: { hits.hits.1._index: test_3 }
+ - match: { hits.hits.1._source.counter: 187.4 }
diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/340_keyword_doc_values.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/340_keyword_doc_values.yml
new file mode 100644
index 0000000000000..8829e7b100fdd
--- /dev/null
+++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/340_keyword_doc_values.yml
@@ -0,0 +1,46 @@
+---
+"search on keyword fields with doc_values enabled":
+ - do:
+ indices.create:
+ index: test
+ body:
+ mappings:
+ properties:
+ "some_keyword":
+ type: "keyword"
+ index: true
+ doc_values: true
+
+ - do:
+ bulk:
+ index: test
+ refresh: true
+ body:
+ - '{"index": {"_index": "test", "_id": "1" }}'
+ - '{ "some_keyword": "ingesting some random keyword data" }'
+ - '{ "index": { "_index": "test", "_id": "2" }}'
+ - '{ "some_keyword": "400" }'
+ - '{ "index": { "_index": "test", "_id": "3" } }'
+ - '{ "some_keyword": "5" }'
+
+ - do:
+ search:
+ index: test
+ body:
+ query:
+ prefix:
+ some_keyword: "ing"
+
+ - match: { hits.hits.0._source.some_keyword: "ingesting some random keyword data" }
+
+ - do:
+ search:
+ index: test
+ body:
+ query:
+ range: {
+ "some_keyword": {
+ "lt": 500
+ } }
+
+ - match: { hits.total.value: 2 }
diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml
index 55e1566656faf..1563daba9de6d 100644
--- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml
+++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml
@@ -320,3 +320,130 @@
- length: { hits.hits: 1 }
- match: { hits.hits.0._index: test }
- match: { hits.hits.0._source.population: null }
+
+---
+"half float":
+ - skip:
+ version: " - 2.10.99"
+ reason: half_float was broken for 2.10 and earlier
+
+ - do:
+ indices.create:
+ index: test
+ body:
+ mappings:
+ properties:
+ population:
+ type: half_float
+ - do:
+ bulk:
+ refresh: true
+ index: test
+ body: |
+ {"index":{}}
+ {"population": 184.4}
+ {"index":{}}
+ {"population": 194.4}
+ {"index":{}}
+ {"population": 144.4}
+ {"index":{}}
+ {"population": 174.4}
+ {"index":{}}
+ {"population": 164.4}
+
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ size: 3
+ sort: [ { population: desc } ]
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 3 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: 194.4 }
+ - match: { hits.hits.1._index: test }
+ - match: { hits.hits.1._source.population: 184.4 }
+ - match: { hits.hits.2._index: test }
+ - match: { hits.hits.2._source.population: 174.4 }
+
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ size: 3
+ sort: [ { population: asc } ]
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 3 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: 144.4 }
+ - match: { hits.hits.1._index: test }
+ - match: { hits.hits.1._source.population: 164.4 }
+ - match: { hits.hits.2._index: test }
+ - match: { hits.hits.2._source.population: 174.4 }
+
+ # search_after with the asc sort
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ size: 1
+ sort: [ { population: asc } ]
+ search_after: [ 184.375 ] # this is rounded sort value in sort result
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 1 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: 194.4 }
+
+ # search_after with the desc sort
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ size: 1
+ sort: [ { population: desc } ]
+ search_after: [ 164.375 ] # this is rounded sort value in sort result
+ - match: { hits.total: 5 }
+ - length: { hits.hits: 1 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: 144.4 }
+
+ # search_after with the asc sort with missing
+ - do:
+ bulk:
+ refresh: true
+ index: test
+ body: |
+ {"index":{}}
+ {"population": null}
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ "size": 5
+ "sort": [ { "population": { "order": "asc", "missing": "_last" } } ]
+ search_after: [ 200 ] # making it out of min/max so only missing value hit is qualified
+
+ - match: { hits.total: 6 }
+ - length: { hits.hits: 1 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: null }
+
+ # search_after with the desc sort with missing
+ - do:
+ search:
+ index: test
+ rest_total_hits_as_int: true
+ body:
+ "size": 5
+ "sort": [ { "population": { "order": "desc", "missing": "_last" } } ]
+ search_after: [ 100 ] # making it out of min/max so only missing value hit is qualified
+
+ - match: { hits.total: 6 }
+ - length: { hits.hits: 1 }
+ - match: { hits.hits.0._index: test }
+ - match: { hits.hits.0._source.population: null }
diff --git a/server/build.gradle b/server/build.gradle
index f6db3d53a0dcc..fa8a44ef6fc94 100644
--- a/server/build.gradle
+++ b/server/build.gradle
@@ -154,6 +154,10 @@ dependencies {
// jcraft
api "com.jcraft:jzlib:${versions.jzlib}"
+ // reactor
+ api "io.projectreactor:reactor-core:${versions.reactor}"
+ api "org.reactivestreams:reactive-streams:${versions.reactivestreams}"
+
// protobuf
api "com.google.protobuf:protobuf-java:${versions.protobuf}"
api "jakarta.annotation:jakarta.annotation-api:${versions.jakarta_annotation}"
@@ -364,11 +368,15 @@ tasks.named("thirdPartyAudit").configure {
'com.google.protobuf.UnsafeUtil$Android32MemoryAccessor',
'com.google.protobuf.UnsafeUtil$Android64MemoryAccessor',
'com.google.protobuf.UnsafeUtil$JvmMemoryAccessor',
- 'com.google.protobuf.UnsafeUtil$MemoryAccessor'
+ 'com.google.protobuf.UnsafeUtil$MemoryAccessor',
+ 'org.apache.logging.log4j.core.util.internal.UnsafeUtil',
+ 'org.apache.logging.log4j.core.util.internal.UnsafeUtil$1',
+ 'reactor.core.publisher.Traces$SharedSecretsCallSiteSupplierFactory$TracingException'
)
}
tasks.named("dependencyLicenses").configure {
+ mapping from: /reactor-.*/, to: 'reactor'
mapping from: /lucene-.*/, to: 'lucene'
dependencies = project.configurations.runtimeClasspath.fileCollection {
it.group.startsWith('org.opensearch') == false ||
diff --git a/server/licenses/log4j-api-2.20.0.jar.sha1 b/server/licenses/log4j-api-2.20.0.jar.sha1
deleted file mode 100644
index 37154d9861ac0..0000000000000
--- a/server/licenses/log4j-api-2.20.0.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-1fe6082e660daf07c689a89c94dc0f49c26b44bb
\ No newline at end of file
diff --git a/server/licenses/log4j-api-2.21.0.jar.sha1 b/server/licenses/log4j-api-2.21.0.jar.sha1
new file mode 100644
index 0000000000000..51446052594aa
--- /dev/null
+++ b/server/licenses/log4j-api-2.21.0.jar.sha1
@@ -0,0 +1 @@
+760192f2b69eacf4a4afc78e5a1d7a8de054fcbd
\ No newline at end of file
diff --git a/server/licenses/log4j-core-2.20.0.jar.sha1 b/server/licenses/log4j-core-2.20.0.jar.sha1
deleted file mode 100644
index 49c972626563b..0000000000000
--- a/server/licenses/log4j-core-2.20.0.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-eb2a9a47b1396e00b5eee1264296729a70565cc0
\ No newline at end of file
diff --git a/server/licenses/log4j-core-2.21.0.jar.sha1 b/server/licenses/log4j-core-2.21.0.jar.sha1
new file mode 100644
index 0000000000000..c88e6f7a25ca9
--- /dev/null
+++ b/server/licenses/log4j-core-2.21.0.jar.sha1
@@ -0,0 +1 @@
+122e1a9e0603cc9eae07b0846a6ff01f2454bc49
\ No newline at end of file
diff --git a/server/licenses/log4j-jul-2.20.0.jar.sha1 b/server/licenses/log4j-jul-2.20.0.jar.sha1
deleted file mode 100644
index a456651e4569e..0000000000000
--- a/server/licenses/log4j-jul-2.20.0.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8170e6118eac1ab332046c179718a0f107f688e1
\ No newline at end of file
diff --git a/server/licenses/log4j-jul-2.21.0.jar.sha1 b/server/licenses/log4j-jul-2.21.0.jar.sha1
new file mode 100644
index 0000000000000..480010840abca
--- /dev/null
+++ b/server/licenses/log4j-jul-2.21.0.jar.sha1
@@ -0,0 +1 @@
+f0da61113f4a47654677e6a98b1e13ca7de2483d
\ No newline at end of file
diff --git a/plugins/crypto-kms/licenses/reactive-streams-1.0.4.jar.sha1 b/server/licenses/reactive-streams-1.0.4.jar.sha1
similarity index 100%
rename from plugins/crypto-kms/licenses/reactive-streams-1.0.4.jar.sha1
rename to server/licenses/reactive-streams-1.0.4.jar.sha1
diff --git a/plugins/crypto-kms/licenses/reactive-streams-LICENSE.txt b/server/licenses/reactive-streams-LICENSE.txt
similarity index 100%
rename from plugins/crypto-kms/licenses/reactive-streams-LICENSE.txt
rename to server/licenses/reactive-streams-LICENSE.txt
diff --git a/plugins/repository-azure/licenses/reactive-streams-NOTICE.txt b/server/licenses/reactive-streams-NOTICE.txt
similarity index 100%
rename from plugins/repository-azure/licenses/reactive-streams-NOTICE.txt
rename to server/licenses/reactive-streams-NOTICE.txt
diff --git a/server/licenses/reactor-LICENSE.txt b/server/licenses/reactor-LICENSE.txt
new file mode 100644
index 0000000000000..e5583c184e67a
--- /dev/null
+++ b/server/licenses/reactor-LICENSE.txt
@@ -0,0 +1,201 @@
+Apache License
+ Version 2.0, January 2004
+ https://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "{}"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright {yyyy} {name of copyright owner}
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ https://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/plugins/repository-s3/licenses/reactive-streams-NOTICE.txt b/server/licenses/reactor-NOTICE.txt
similarity index 100%
rename from plugins/repository-s3/licenses/reactive-streams-NOTICE.txt
rename to server/licenses/reactor-NOTICE.txt
diff --git a/server/licenses/reactor-core-3.5.11.jar.sha1 b/server/licenses/reactor-core-3.5.11.jar.sha1
new file mode 100644
index 0000000000000..e5ffdbc8a7840
--- /dev/null
+++ b/server/licenses/reactor-core-3.5.11.jar.sha1
@@ -0,0 +1 @@
+db2299757f562261eb775d13658e86ff06f91e8a
\ No newline at end of file
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/HotThreadsIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/HotThreadsIT.java
index 6343bd127c458..4c9f49df71257 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/HotThreadsIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/HotThreadsIT.java
@@ -60,8 +60,8 @@
public class HotThreadsIT extends OpenSearchIntegTestCase {
public void testHotThreadsDontFail() throws ExecutionException, InterruptedException {
- /**
- * This test just checks if nothing crashes or gets stuck etc.
+ /*
+ This test just checks if nothing crashes or gets stuck etc.
*/
createIndex("test");
final int iters = scaledRandomIntBetween(2, 20);
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java
index 0197ccf059737..44ba585016d8e 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java
@@ -112,7 +112,7 @@ protected int numberOfEvents(String actionMasks, Function findEvents(String actionMasks, Function, Boolean> criteria) {
List events = new ArrayList<>();
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/ConcurrentSearchTasksIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/ConcurrentSearchTasksIT.java
index c733329a1b5f7..e6fd9139d45f2 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/ConcurrentSearchTasksIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/ConcurrentSearchTasksIT.java
@@ -30,7 +30,7 @@
/**
* Integration tests for task management API with Concurrent Segment Search
- *
+ *
* The way the test framework bootstraps the test cluster makes it difficult to parameterize the feature flag.
* Once concurrent search is moved behind a cluster setting we can parameterize these tests behind the setting.
*/
@@ -72,7 +72,7 @@ protected Settings featureFlagSettings() {
/**
* Tests the number of threads that worked on a search task.
- *
+ *
* Currently, we try to control concurrency by creating an index with 7 segments and rely on
* the way concurrent search creates leaf slices from segments. Once more concurrency controls are introduced
* we should improve this test to use those methods.
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/repositories/RepositoryBlocksIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/repositories/RepositoryBlocksIT.java
index aff7c5d9876ac..36fe3748e9d10 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/repositories/RepositoryBlocksIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/repositories/RepositoryBlocksIT.java
@@ -46,7 +46,7 @@
/**
* This class tests that repository operations (Put, Delete, Verify) are blocked when the cluster is read-only.
- *
+ *
* The @NodeScope TEST is needed because this class updates the cluster setting "cluster.blocks.read_only".
*/
@ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST)
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java
index 347011721c728..78fb01b07b6b1 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/snapshots/SnapshotBlocksIT.java
@@ -53,7 +53,7 @@
/**
* This class tests that snapshot operations (Create, Delete, Restore) are blocked when the cluster is read-only.
- *
+ *
* The @NodeScope TEST is needed because this class updates the cluster setting "cluster.blocks.read_only".
*/
@ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST)
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java
new file mode 100644
index 0000000000000..a081110e6c5a1
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java
@@ -0,0 +1,133 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.action.admin.indices.create;
+
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+import org.opensearch.Version;
+import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse;
+import org.opensearch.action.admin.indices.shrink.ResizeType;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.index.query.TermsQueryBuilder;
+import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase;
+import org.opensearch.test.VersionUtils;
+
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
+import static org.hamcrest.Matchers.equalTo;
+
+public class RemoteCloneIndexIT extends RemoteStoreBaseIntegTestCase {
+
+ @Override
+ protected boolean forbidPrivateIndexSettings() {
+ return false;
+ }
+
+ public void testCreateCloneIndex() {
+ Version version = VersionUtils.randomIndexCompatibleVersion(random());
+ int numPrimaryShards = randomIntBetween(1, 5);
+ prepareCreate("source").setSettings(
+ Settings.builder().put(indexSettings()).put("number_of_shards", numPrimaryShards).put("index.version.created", version)
+ ).get();
+ final int docs = randomIntBetween(0, 128);
+ for (int i = 0; i < docs; i++) {
+ client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+ // relocate all shards to one node such that we can merge it.
+ client().admin().indices().prepareUpdateSettings("source").setSettings(Settings.builder().put("index.blocks.write", true)).get();
+ ensureGreen();
+
+ final IndicesStatsResponse sourceStats = client().admin().indices().prepareStats("source").setSegments(true).get();
+
+ // disable rebalancing to be able to capture the right stats. balancing can move the target primary
+ // making it hard to pin point the source shards.
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none"))
+ .get();
+ try {
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setResizeType(ResizeType.CLONE)
+ .setSettings(Settings.builder().put("index.number_of_replicas", 0).putNull("index.blocks.write").build())
+ .get()
+ );
+ ensureGreen();
+
+ final IndicesStatsResponse targetStats = client().admin().indices().prepareStats("target").get();
+ assertThat(targetStats.getIndex("target").getIndexShards().keySet().size(), equalTo(numPrimaryShards));
+
+ final int size = docs > 0 ? 2 * docs : 1;
+ assertHitCount(client().prepareSearch("target").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+
+ for (int i = docs; i < 2 * docs; i++) {
+ client().prepareIndex("target").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ flushAndRefresh();
+ assertHitCount(
+ client().prepareSearch("target").setSize(2 * size).setQuery(new TermsQueryBuilder("foo", "bar")).get(),
+ 2 * docs
+ );
+ assertHitCount(client().prepareSearch("source").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+ GetSettingsResponse target = client().admin().indices().prepareGetSettings("target").get();
+ assertEquals(version, target.getIndexToSettings().get("target").getAsVersion("index.version.created", null));
+ } finally {
+ // clean up
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(
+ Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), (String) null)
+ )
+ .get();
+ }
+
+ }
+
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java
new file mode 100644
index 0000000000000..282eb9c6ad95e
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java
@@ -0,0 +1,545 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.action.admin.indices.create;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.util.Constants;
+import org.opensearch.Version;
+import org.opensearch.action.admin.cluster.reroute.ClusterRerouteResponse;
+import org.opensearch.action.admin.cluster.state.ClusterStateRequest;
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse;
+import org.opensearch.action.admin.indices.stats.CommonStats;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.action.admin.indices.stats.ShardStats;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.support.ActiveShardCount;
+import org.opensearch.client.Client;
+import org.opensearch.cluster.ClusterInfoService;
+import org.opensearch.cluster.ClusterState;
+import org.opensearch.cluster.InternalClusterInfoService;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.cluster.node.DiscoveryNode;
+import org.opensearch.cluster.routing.Murmur3HashFunction;
+import org.opensearch.cluster.routing.RoutingTable;
+import org.opensearch.cluster.routing.ShardRouting;
+import org.opensearch.cluster.routing.UnassignedInfo;
+import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider;
+import org.opensearch.common.Priority;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.index.Index;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.index.IndexModule;
+import org.opensearch.index.IndexService;
+import org.opensearch.index.engine.SegmentsStats;
+import org.opensearch.index.query.TermsQueryBuilder;
+import org.opensearch.index.seqno.SeqNoStats;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.IndicesService;
+import org.opensearch.indices.replication.common.ReplicationType;
+import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase;
+import org.opensearch.test.VersionUtils;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+
+public class RemoteShrinkIndexIT extends RemoteStoreBaseIntegTestCase {
+ @Override
+ protected boolean forbidPrivateIndexSettings() {
+ return false;
+ }
+
+ public Settings indexSettings() {
+ return Settings.builder()
+ .put(super.indexSettings())
+ .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
+ .put(SETTING_NUMBER_OF_REPLICAS, 0)
+ .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
+ .build();
+ }
+
+ public void testCreateShrinkIndexToN() {
+
+ assumeFalse("https://github.com/elastic/elasticsearch/issues/34080", Constants.WINDOWS);
+
+ int[][] possibleShardSplits = new int[][] { { 8, 4, 2 }, { 9, 3, 1 }, { 4, 2, 1 }, { 15, 5, 1 } };
+ int[] shardSplits = randomFrom(possibleShardSplits);
+ assertEquals(shardSplits[0], (shardSplits[0] / shardSplits[1]) * shardSplits[1]);
+ assertEquals(shardSplits[1], (shardSplits[1] / shardSplits[2]) * shardSplits[2]);
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", shardSplits[0])).get();
+ for (int i = 0; i < 20; i++) {
+ client().prepareIndex("source")
+ .setId(Integer.toString(i))
+ .setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON)
+ .get();
+ }
+ final Map dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
+ assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
+ DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(new DiscoveryNode[0]);
+ String mergeNode = discoveryNodes[0].getName();
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+ // relocate all shards to one node such that we can merge it.
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source")
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode).put("index.blocks.write", true))
+ .get();
+ ensureGreen();
+ // now merge source into a 4 shard index
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "first_shrink")
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", shardSplits[1])
+ .putNull("index.blocks.write")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+ assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+
+ for (int i = 0; i < 20; i++) { // now update
+ client().prepareIndex("first_shrink")
+ .setId(Integer.toString(i))
+ .setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON)
+ .get();
+ }
+ flushAndRefresh();
+ assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+
+ // relocate all shards to one node such that we can merge it.
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("first_shrink")
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode).put("index.blocks.write", true))
+ .get();
+ ensureGreen();
+ // now merge source into a 2 shard index
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("first_shrink", "second_shrink")
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", shardSplits[2])
+ .putNull("index.blocks.write")
+ .putNull("index.routing.allocation.require._name")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+ assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ // let it be allocated anywhere and bump replicas
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("second_shrink")
+ .setSettings(Settings.builder().putNull("index.routing.allocation.include._id").put("index.number_of_replicas", 0))
+ .get();
+ ensureGreen();
+ assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+
+ for (int i = 0; i < 20; i++) { // now update
+ client().prepareIndex("second_shrink")
+ .setId(Integer.toString(i))
+ .setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON)
+ .get();
+ }
+ flushAndRefresh();
+ assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ }
+
+ public void testShrinkIndexPrimaryTerm() throws Exception {
+ int numberOfShards = randomIntBetween(2, 20);
+ int numberOfTargetShards = randomValueOtherThanMany(n -> numberOfShards % n != 0, () -> randomIntBetween(1, numberOfShards - 1));
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", numberOfShards)).get();
+
+ final Map dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
+ assertThat(dataNodes.size(), greaterThanOrEqualTo(2));
+ final DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(new DiscoveryNode[0]);
+ final String mergeNode = discoveryNodes[0].getName();
+ // This needs more than the default timeout if a large number of shards were created.
+ ensureGreen(TimeValue.timeValueSeconds(120));
+
+ // fail random primary shards to force primary terms to increase
+ final Index source = resolveIndex("source");
+ final int iterations = scaledRandomIntBetween(0, 16);
+ for (int i = 0; i < iterations; i++) {
+ final String node = randomSubsetOf(1, internalCluster().nodesInclude("source")).get(0);
+ final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node);
+ final IndexService indexShards = indexServices.indexServiceSafe(source);
+ for (final Integer shardId : indexShards.shardIds()) {
+ final IndexShard shard = indexShards.getShard(shardId);
+ if (shard.routingEntry().primary() && randomBoolean()) {
+ disableAllocation("source");
+ shard.failShard("test", new Exception("test"));
+ // this can not succeed until the shard is failed and a replica is promoted
+ int id = 0;
+ while (true) {
+ // find an ID that routes to the right shard, we will only index to the shard that saw a primary failure
+ final String s = Integer.toString(id);
+ final int hash = Math.floorMod(Murmur3HashFunction.hash(s), numberOfShards);
+ if (hash == shardId) {
+ final IndexRequest request = new IndexRequest("source").id(s)
+ .source("{ \"f\": \"" + s + "\"}", MediaTypeRegistry.JSON);
+ client().index(request).get();
+ break;
+ } else {
+ id++;
+ }
+ }
+ enableAllocation("source");
+ ensureGreen();
+ }
+ }
+ }
+
+ // relocate all shards to one node such that we can merge it.
+ final Settings.Builder prepareShrinkSettings = Settings.builder()
+ .put("index.routing.allocation.require._name", mergeNode)
+ .put("index.blocks.write", true);
+ client().admin().indices().prepareUpdateSettings("source").setSettings(prepareShrinkSettings).get();
+ ensureGreen(TimeValue.timeValueSeconds(120)); // needs more than the default to relocate many shards
+
+ final IndexMetadata indexMetadata = indexMetadata(client(), "source");
+ final long beforeShrinkPrimaryTerm = IntStream.range(0, numberOfShards).mapToLong(indexMetadata::primaryTerm).max().getAsLong();
+
+ // now merge source into target
+ final Settings shrinkSettings = Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", numberOfTargetShards)
+ .build();
+ assertAcked(client().admin().indices().prepareResizeIndex("source", "target").setSettings(shrinkSettings).get());
+
+ ensureGreen(TimeValue.timeValueSeconds(120));
+
+ final IndexMetadata afterShrinkIndexMetadata = indexMetadata(client(), "target");
+ for (int shardId = 0; shardId < numberOfTargetShards; shardId++) {
+ assertThat(afterShrinkIndexMetadata.primaryTerm(shardId), equalTo(beforeShrinkPrimaryTerm + 1));
+ }
+ }
+
+ private static IndexMetadata indexMetadata(final Client client, final String index) {
+ final ClusterStateResponse clusterStateResponse = client.admin().cluster().state(new ClusterStateRequest()).actionGet();
+ return clusterStateResponse.getState().metadata().index(index);
+ }
+
+ public void testCreateShrinkIndex() {
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ Version version = VersionUtils.randomVersion(random());
+ prepareCreate("source").setSettings(
+ Settings.builder().put(indexSettings()).put("number_of_shards", randomIntBetween(2, 7)).put("index.version.created", version)
+ ).get();
+ final int docs = randomIntBetween(0, 128);
+ for (int i = 0; i < docs; i++) {
+ client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ final Map dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
+ assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
+ DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(new DiscoveryNode[0]);
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+ // relocate all shards to one node such that we can merge it.
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source")
+ .setSettings(
+ Settings.builder()
+ .put("index.routing.allocation.require._name", discoveryNodes[0].getName())
+ .put("index.blocks.write", true)
+ )
+ .get();
+ ensureGreen();
+
+ final IndicesStatsResponse sourceStats = client().admin().indices().prepareStats("source").setSegments(true).get();
+
+ // disable rebalancing to be able to capture the right stats. balancing can move the target primary
+ // making it hard to pin point the source shards.
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none"))
+ .get();
+
+ // now merge source into a single shard index
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .putNull("index.blocks.write")
+ .putNull("index.routing.allocation.require._name")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+
+ // resolve true merge node - this is not always the node we required as all shards may be on another node
+ final ClusterState state = client().admin().cluster().prepareState().get().getState();
+ DiscoveryNode mergeNode = state.nodes().get(state.getRoutingTable().index("target").shard(0).primaryShard().currentNodeId());
+ logger.info("merge node {}", mergeNode);
+
+ final long maxSeqNo = Arrays.stream(sourceStats.getShards())
+ .filter(shard -> shard.getShardRouting().currentNodeId().equals(mergeNode.getId()))
+ .map(ShardStats::getSeqNoStats)
+ .mapToLong(SeqNoStats::getMaxSeqNo)
+ .max()
+ .getAsLong();
+ final long maxUnsafeAutoIdTimestamp = Arrays.stream(sourceStats.getShards())
+ .filter(shard -> shard.getShardRouting().currentNodeId().equals(mergeNode.getId()))
+ .map(ShardStats::getStats)
+ .map(CommonStats::getSegments)
+ .mapToLong(SegmentsStats::getMaxUnsafeAutoIdTimestamp)
+ .max()
+ .getAsLong();
+
+ final IndicesStatsResponse targetStats = client().admin().indices().prepareStats("target").get();
+ for (final ShardStats shardStats : targetStats.getShards()) {
+ final SeqNoStats seqNoStats = shardStats.getSeqNoStats();
+ final ShardRouting shardRouting = shardStats.getShardRouting();
+ assertThat("failed on " + shardRouting, seqNoStats.getMaxSeqNo(), equalTo(maxSeqNo));
+ assertThat("failed on " + shardRouting, seqNoStats.getLocalCheckpoint(), equalTo(maxSeqNo));
+ assertThat(
+ "failed on " + shardRouting,
+ shardStats.getStats().getSegments().getMaxUnsafeAutoIdTimestamp(),
+ equalTo(maxUnsafeAutoIdTimestamp)
+ );
+ }
+
+ final int size = docs > 0 ? 2 * docs : 1;
+ assertHitCount(client().prepareSearch("target").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+
+ for (int i = docs; i < 2 * docs; i++) {
+ client().prepareIndex("target").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ flushAndRefresh();
+ assertHitCount(client().prepareSearch("target").setSize(2 * size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 2 * docs);
+ assertHitCount(client().prepareSearch("source").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+ GetSettingsResponse target = client().admin().indices().prepareGetSettings("target").get();
+ assertEquals(version, target.getIndexToSettings().get("target").getAsVersion("index.version.created", null));
+
+ // clean up
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(
+ Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), (String) null)
+ )
+ .get();
+ }
+
+ /**
+ * Tests that we can manually recover from a failed allocation due to shards being moved away etc.
+ */
+ public void testCreateShrinkIndexFails() throws Exception {
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ prepareCreate("source").setSettings(
+ Settings.builder().put(indexSettings()).put("number_of_shards", randomIntBetween(2, 7)).put("number_of_replicas", 0)
+ ).get();
+ for (int i = 0; i < 20; i++) {
+ client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ final Map dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
+ assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
+ DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(new DiscoveryNode[0]);
+ String spareNode = discoveryNodes[0].getName();
+ String mergeNode = discoveryNodes[1].getName();
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+ // relocate all shards to one node such that we can merge it.
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source")
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode).put("index.blocks.write", true))
+ .get();
+ ensureGreen();
+
+ // now merge source into a single shard index
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setWaitForActiveShards(ActiveShardCount.NONE)
+ .setSettings(
+ Settings.builder()
+ .put("index.routing.allocation.exclude._name", mergeNode) // we manually exclude the merge node to forcefully fuck it up
+ .put("index.number_of_replicas", 0)
+ .put("index.allocation.max_retries", 1)
+ .build()
+ )
+ .get();
+ client().admin().cluster().prepareHealth("target").setWaitForEvents(Priority.LANGUID).get();
+
+ // now we move all shards away from the merge node
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source")
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", spareNode).put("index.blocks.write", true))
+ .get();
+ ensureGreen("source");
+
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("target") // erase the forcefully fuckup!
+ .setSettings(Settings.builder().putNull("index.routing.allocation.exclude._name"))
+ .get();
+ // wait until it fails
+ assertBusy(() -> {
+ ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
+ RoutingTable routingTables = clusterStateResponse.getState().routingTable();
+ assertTrue(routingTables.index("target").shard(0).getShards().get(0).unassigned());
+ assertEquals(
+ UnassignedInfo.Reason.ALLOCATION_FAILED,
+ routingTables.index("target").shard(0).getShards().get(0).unassignedInfo().getReason()
+ );
+ assertEquals(1, routingTables.index("target").shard(0).getShards().get(0).unassignedInfo().getNumFailedAllocations());
+ });
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source") // now relocate them all to the right node
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode))
+ .get();
+ ensureGreen("source");
+
+ final InternalClusterInfoService infoService = (InternalClusterInfoService) internalCluster().getInstance(
+ ClusterInfoService.class,
+ internalCluster().getClusterManagerName()
+ );
+ infoService.refresh();
+ // kick off a retry and wait until it's done!
+ ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get();
+ long expectedShardSize = clusterRerouteResponse.getState()
+ .routingTable()
+ .index("target")
+ .shard(0)
+ .getShards()
+ .get(0)
+ .getExpectedShardSize();
+ // we support the expected shard size in the allocator to sum up over the source index shards
+ assertTrue("expected shard size must be set but wasn't: " + expectedShardSize, expectedShardSize > 0);
+ ensureGreen();
+ assertHitCount(client().prepareSearch("target").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
+ }
+
+ public void testCreateShrinkWithIndexSort() throws Exception {
+ SortField expectedSortField = new SortedSetSortField("id", true, SortedSetSelector.Type.MAX);
+ expectedSortField.setMissingValue(SortedSetSortField.STRING_FIRST);
+ Sort expectedIndexSort = new Sort(expectedSortField);
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ prepareCreate("source").setSettings(
+ Settings.builder()
+ .put(indexSettings())
+ .put("sort.field", "id")
+ .put("sort.order", "desc")
+ .put("number_of_shards", 8)
+ .put("number_of_replicas", 0)
+ ).setMapping("id", "type=keyword,doc_values=true").get();
+ for (int i = 0; i < 20; i++) {
+ client().prepareIndex("source")
+ .setId(Integer.toString(i))
+ .setSource("{\"foo\" : \"bar\", \"id\" : " + i + "}", MediaTypeRegistry.JSON)
+ .get();
+ }
+ final Map dataNodes = client().admin().cluster().prepareState().get().getState().nodes().getDataNodes();
+ assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
+ DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(new DiscoveryNode[0]);
+ String mergeNode = discoveryNodes[0].getName();
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+
+ flushAndRefresh();
+ assertSortedSegments("source", expectedIndexSort);
+
+ // relocate all shards to one node such that we can merge it.
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("source")
+ .setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode).put("index.blocks.write", true))
+ .get();
+ ensureGreen();
+
+ // check that index sort cannot be set on the target index
+ IllegalArgumentException exc = expectThrows(
+ IllegalArgumentException.class,
+ () -> client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", "2")
+ .put("index.sort.field", "foo")
+ .build()
+ )
+ .get()
+ );
+ assertThat(exc.getMessage(), containsString("can't override index sort when resizing an index"));
+
+ // check that the index sort order of `source` is correctly applied to the `target`
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", "2")
+ .putNull("index.blocks.write")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+ flushAndRefresh();
+ GetSettingsResponse settingsResponse = client().admin().indices().prepareGetSettings("target").execute().actionGet();
+ assertEquals(settingsResponse.getSetting("target", "index.sort.field"), "id");
+ assertEquals(settingsResponse.getSetting("target", "index.sort.order"), "desc");
+ assertSortedSegments("target", expectedIndexSort);
+
+ // ... and that the index sort is also applied to updates
+ for (int i = 20; i < 40; i++) {
+ client().prepareIndex("target").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ flushAndRefresh();
+ assertSortedSegments("target", expectedIndexSort);
+ }
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java
new file mode 100644
index 0000000000000..dd4252d24f314
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java
@@ -0,0 +1,506 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.action.admin.indices.create;
+
+import org.apache.lucene.search.join.ScoreMode;
+import org.apache.lucene.util.Constants;
+import org.opensearch.Version;
+import org.opensearch.action.admin.cluster.state.ClusterStateRequest;
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse;
+import org.opensearch.action.admin.indices.shrink.ResizeType;
+import org.opensearch.action.admin.indices.stats.CommonStats;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.action.admin.indices.stats.ShardStats;
+import org.opensearch.action.get.GetResponse;
+import org.opensearch.action.index.IndexRequest;
+import org.opensearch.action.index.IndexRequestBuilder;
+import org.opensearch.action.search.SearchResponse;
+import org.opensearch.client.Client;
+import org.opensearch.cluster.ClusterState;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.cluster.metadata.MetadataCreateIndexService;
+import org.opensearch.cluster.node.DiscoveryNode;
+import org.opensearch.cluster.routing.Murmur3HashFunction;
+import org.opensearch.cluster.routing.ShardRouting;
+import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.index.Index;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.index.IndexModule;
+import org.opensearch.index.IndexService;
+import org.opensearch.index.engine.SegmentsStats;
+import org.opensearch.index.query.TermsQueryBuilder;
+import org.opensearch.index.seqno.SeqNoStats;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.IndicesService;
+import org.opensearch.indices.replication.common.ReplicationType;
+import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase;
+import org.opensearch.test.VersionUtils;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+
+import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
+import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder;
+import static org.opensearch.index.query.QueryBuilders.nestedQuery;
+import static org.opensearch.index.query.QueryBuilders.termQuery;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures;
+import static org.hamcrest.Matchers.equalTo;
+
+public class RemoteSplitIndexIT extends RemoteStoreBaseIntegTestCase {
+
+ @Override
+ protected boolean forbidPrivateIndexSettings() {
+ return false;
+ }
+
+ public Settings indexSettings() {
+ return Settings.builder()
+ .put(super.indexSettings())
+ .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false)
+ .put(SETTING_NUMBER_OF_REPLICAS, 0)
+ .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
+ .build();
+ }
+
+ public void testCreateSplitIndexToN() throws IOException {
+ int[][] possibleShardSplits = new int[][] { { 2, 4, 8 }, { 3, 6, 12 }, { 1, 2, 4 } };
+ int[] shardSplits = randomFrom(possibleShardSplits);
+ splitToN(shardSplits[0], shardSplits[1], shardSplits[2]);
+ }
+
+ public void testSplitFromOneToN() {
+
+ assumeFalse("https://github.com/elastic/elasticsearch/issues/34080", Constants.WINDOWS);
+
+ splitToN(1, 5, 10);
+ client().admin().indices().prepareDelete("*").get();
+ int randomSplit = randomIntBetween(2, 6);
+ splitToN(1, randomSplit, randomSplit * 2);
+ }
+
+ private void splitToN(int sourceShards, int firstSplitShards, int secondSplitShards) {
+
+ assertEquals(sourceShards, (sourceShards * firstSplitShards) / firstSplitShards);
+ assertEquals(firstSplitShards, (firstSplitShards * secondSplitShards) / secondSplitShards);
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ final boolean useRouting = randomBoolean();
+ final boolean useNested = randomBoolean();
+ final boolean useMixedRouting = useRouting ? randomBoolean() : false;
+ CreateIndexRequestBuilder createInitialIndex = prepareCreate("source");
+ Settings.Builder settings = Settings.builder().put(indexSettings()).put("number_of_shards", sourceShards);
+ final boolean useRoutingPartition;
+ if (randomBoolean()) {
+ // randomly set the value manually
+ int routingShards = secondSplitShards * randomIntBetween(1, 10);
+ settings.put("index.number_of_routing_shards", routingShards);
+ useRoutingPartition = false;
+ } else {
+ useRoutingPartition = randomBoolean();
+ }
+ if (useRouting && useMixedRouting == false && useRoutingPartition) {
+ int numRoutingShards = MetadataCreateIndexService.calculateNumRoutingShards(secondSplitShards, Version.CURRENT) - 1;
+ settings.put("index.routing_partition_size", randomIntBetween(1, numRoutingShards));
+ if (useNested) {
+ createInitialIndex.setMapping("_routing", "required=true", "nested1", "type=nested");
+ } else {
+ createInitialIndex.setMapping("_routing", "required=true");
+ }
+ } else if (useNested) {
+ createInitialIndex.setMapping("nested1", "type=nested");
+ }
+ logger.info("use routing {} use mixed routing {} use nested {}", useRouting, useMixedRouting, useNested);
+ createInitialIndex.setSettings(settings).get();
+
+ int numDocs = randomIntBetween(10, 50);
+ String[] routingValue = new String[numDocs];
+
+ BiFunction indexFunc = (index, id) -> {
+ try {
+ return client().prepareIndex(index)
+ .setId(Integer.toString(id))
+ .setSource(
+ jsonBuilder().startObject()
+ .field("foo", "bar")
+ .field("i", id)
+ .startArray("nested1")
+ .startObject()
+ .field("n_field1", "n_value1_1")
+ .field("n_field2", "n_value2_1")
+ .endObject()
+ .startObject()
+ .field("n_field1", "n_value1_2")
+ .field("n_field2", "n_value2_2")
+ .endObject()
+ .endArray()
+ .endObject()
+ );
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ };
+ for (int i = 0; i < numDocs; i++) {
+ IndexRequestBuilder builder = indexFunc.apply("source", i);
+ if (useRouting) {
+ String routing = randomRealisticUnicodeOfCodepointLengthBetween(1, 10);
+ if (useMixedRouting && randomBoolean()) {
+ routingValue[i] = null;
+ } else {
+ routingValue[i] = routing;
+ }
+ builder.setRouting(routingValue[i]);
+ }
+ builder.get();
+ }
+
+ if (randomBoolean()) {
+ for (int i = 0; i < numDocs; i++) { // let's introduce some updates / deletes on the index
+ if (randomBoolean()) {
+ IndexRequestBuilder builder = indexFunc.apply("source", i);
+ if (useRouting) {
+ builder.setRouting(routingValue[i]);
+ }
+ builder.get();
+ }
+ }
+ }
+
+ ensureYellow();
+ client().admin().indices().prepareUpdateSettings("source").setSettings(Settings.builder().put("index.blocks.write", true)).get();
+ ensureGreen();
+ Settings.Builder firstSplitSettingsBuilder = Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", firstSplitShards)
+ .putNull("index.blocks.write");
+ if (sourceShards == 1 && useRoutingPartition == false && randomBoolean()) { // try to set it if we have a source index with 1 shard
+ firstSplitSettingsBuilder.put("index.number_of_routing_shards", secondSplitShards);
+ }
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "first_split")
+ .setResizeType(ResizeType.SPLIT)
+ .setSettings(firstSplitSettingsBuilder.build())
+ .get()
+ );
+ ensureGreen();
+ assertHitCount(client().prepareSearch("first_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+
+ for (int i = 0; i < numDocs; i++) { // now update
+ IndexRequestBuilder builder = indexFunc.apply("first_split", i);
+ if (useRouting) {
+ builder.setRouting(routingValue[i]);
+ }
+ builder.get();
+ }
+ flushAndRefresh();
+ assertHitCount(client().prepareSearch("first_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ for (int i = 0; i < numDocs; i++) {
+ GetResponse getResponse = client().prepareGet("first_split", Integer.toString(i)).setRouting(routingValue[i]).get();
+ assertTrue(getResponse.isExists());
+ }
+
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("first_split")
+ .setSettings(Settings.builder().put("index.blocks.write", true))
+ .get();
+ ensureGreen();
+ // now split source into a new index
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("first_split", "second_split")
+ .setResizeType(ResizeType.SPLIT)
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", secondSplitShards)
+ .putNull("index.blocks.write")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+ assertHitCount(client().prepareSearch("second_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ // let it be allocated anywhere and bump replicas
+ client().admin()
+ .indices()
+ .prepareUpdateSettings("second_split")
+ .setSettings(Settings.builder().put("index.number_of_replicas", 0))
+ .get();
+ ensureGreen();
+ assertHitCount(client().prepareSearch("second_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+
+ for (int i = 0; i < numDocs; i++) { // now update
+ IndexRequestBuilder builder = indexFunc.apply("second_split", i);
+ if (useRouting) {
+ builder.setRouting(routingValue[i]);
+ }
+ builder.get();
+ }
+ flushAndRefresh();
+ for (int i = 0; i < numDocs; i++) {
+ GetResponse getResponse = client().prepareGet("second_split", Integer.toString(i)).setRouting(routingValue[i]).get();
+ assertTrue(getResponse.isExists());
+ }
+ assertHitCount(client().prepareSearch("second_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ assertHitCount(client().prepareSearch("first_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ if (useNested) {
+ assertNested("source", numDocs);
+ assertNested("first_split", numDocs);
+ assertNested("second_split", numDocs);
+ }
+ assertAllUniqueDocs(
+ client().prepareSearch("second_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(),
+ numDocs
+ );
+ assertAllUniqueDocs(
+ client().prepareSearch("first_split").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(),
+ numDocs
+ );
+ assertAllUniqueDocs(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), numDocs);
+ }
+
+ public void assertNested(String index, int numDocs) {
+ // now, do a nested query
+ SearchResponse searchResponse = client().prepareSearch(index)
+ .setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1_1"), ScoreMode.Avg))
+ .get();
+ assertNoFailures(searchResponse);
+ assertThat(searchResponse.getHits().getTotalHits().value, equalTo((long) numDocs));
+ }
+
+ public void assertAllUniqueDocs(SearchResponse response, int numDocs) {
+ Set ids = new HashSet<>();
+ for (int i = 0; i < response.getHits().getHits().length; i++) {
+ String id = response.getHits().getHits()[i].getId();
+ assertTrue("found ID " + id + " more than once", ids.add(id));
+ }
+ assertEquals(numDocs, ids.size());
+ }
+
+ public void testSplitIndexPrimaryTerm() throws Exception {
+ int numberOfTargetShards = randomIntBetween(2, 20);
+ int numberOfShards = randomValueOtherThanMany(n -> numberOfTargetShards % n != 0, () -> between(1, numberOfTargetShards - 1));
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ prepareCreate("source").setSettings(
+ Settings.builder()
+ .put(indexSettings())
+ .put("number_of_shards", numberOfShards)
+ .put("index.number_of_routing_shards", numberOfTargetShards)
+ ).get();
+ ensureGreen(TimeValue.timeValueSeconds(120)); // needs more than the default to allocate many shards
+
+ // fail random primary shards to force primary terms to increase
+ final Index source = resolveIndex("source");
+ final int iterations = scaledRandomIntBetween(0, 16);
+ for (int i = 0; i < iterations; i++) {
+ final String node = randomSubsetOf(1, internalCluster().nodesInclude("source")).get(0);
+ final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node);
+ final IndexService indexShards = indexServices.indexServiceSafe(source);
+ for (final Integer shardId : indexShards.shardIds()) {
+ final IndexShard shard = indexShards.getShard(shardId);
+ if (shard.routingEntry().primary() && randomBoolean()) {
+ disableAllocation("source");
+ shard.failShard("test", new Exception("test"));
+ // this can not succeed until the shard is failed and a replica is promoted
+ int id = 0;
+ while (true) {
+ // find an ID that routes to the right shard, we will only index to the shard that saw a primary failure
+ final String s = Integer.toString(id);
+ final int hash = Math.floorMod(Murmur3HashFunction.hash(s), numberOfShards);
+ if (hash == shardId) {
+ final IndexRequest request = new IndexRequest("source").id(s)
+ .source("{ \"f\": \"" + s + "\"}", MediaTypeRegistry.JSON);
+ client().index(request).get();
+ break;
+ } else {
+ id++;
+ }
+ }
+ enableAllocation("source");
+ ensureGreen();
+ }
+ }
+ }
+
+ final Settings.Builder prepareSplitSettings = Settings.builder().put("index.blocks.write", true);
+ client().admin().indices().prepareUpdateSettings("source").setSettings(prepareSplitSettings).get();
+ ensureYellow();
+
+ final IndexMetadata indexMetadata = indexMetadata(client(), "source");
+ final long beforeSplitPrimaryTerm = IntStream.range(0, numberOfShards).mapToLong(indexMetadata::primaryTerm).max().getAsLong();
+
+ // now split source into target
+ final Settings splitSettings = Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", numberOfTargetShards)
+ .putNull("index.blocks.write")
+ .build();
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setResizeType(ResizeType.SPLIT)
+ .setSettings(splitSettings)
+ .get()
+ );
+
+ ensureGreen(TimeValue.timeValueSeconds(120)); // needs more than the default to relocate many shards
+
+ final IndexMetadata aftersplitIndexMetadata = indexMetadata(client(), "target");
+ for (int shardId = 0; shardId < numberOfTargetShards; shardId++) {
+ assertThat(aftersplitIndexMetadata.primaryTerm(shardId), equalTo(beforeSplitPrimaryTerm + 1));
+ }
+ }
+
+ private static IndexMetadata indexMetadata(final Client client, final String index) {
+ final ClusterStateResponse clusterStateResponse = client.admin().cluster().state(new ClusterStateRequest()).actionGet();
+ return clusterStateResponse.getState().metadata().index(index);
+ }
+
+ public void testCreateSplitIndex() throws Exception {
+ internalCluster().ensureAtLeastNumDataNodes(2);
+ Version version = VersionUtils.randomIndexCompatibleVersion(random());
+ prepareCreate("source").setSettings(
+ Settings.builder().put(indexSettings()).put("number_of_shards", 1).put("index.version.created", version)
+ ).get();
+ final int docs = randomIntBetween(0, 128);
+ for (int i = 0; i < docs; i++) {
+ client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
+ // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
+ // to the require._name below.
+ ensureGreen();
+ // relocate all shards to one node such that we can merge it.
+ client().admin().indices().prepareUpdateSettings("source").setSettings(Settings.builder().put("index.blocks.write", true)).get();
+ ensureGreen();
+
+ final IndicesStatsResponse sourceStats = client().admin().indices().prepareStats("source").setSegments(true).get();
+
+ // disable rebalancing to be able to capture the right stats. balancing can move the target primary
+ // making it hard to pin point the source shards.
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none"))
+ .get();
+ try {
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareResizeIndex("source", "target")
+ .setResizeType(ResizeType.SPLIT)
+ .setSettings(
+ Settings.builder()
+ .put("index.number_of_replicas", 0)
+ .put("index.number_of_shards", 2)
+ .putNull("index.blocks.write")
+ .build()
+ )
+ .get()
+ );
+ ensureGreen();
+
+ final ClusterState state = client().admin().cluster().prepareState().get().getState();
+ DiscoveryNode mergeNode = state.nodes().get(state.getRoutingTable().index("target").shard(0).primaryShard().currentNodeId());
+ logger.info("split node {}", mergeNode);
+
+ final long maxSeqNo = Arrays.stream(sourceStats.getShards())
+ .filter(shard -> shard.getShardRouting().currentNodeId().equals(mergeNode.getId()))
+ .map(ShardStats::getSeqNoStats)
+ .mapToLong(SeqNoStats::getMaxSeqNo)
+ .max()
+ .getAsLong();
+ final long maxUnsafeAutoIdTimestamp = Arrays.stream(sourceStats.getShards())
+ .filter(shard -> shard.getShardRouting().currentNodeId().equals(mergeNode.getId()))
+ .map(ShardStats::getStats)
+ .map(CommonStats::getSegments)
+ .mapToLong(SegmentsStats::getMaxUnsafeAutoIdTimestamp)
+ .max()
+ .getAsLong();
+
+ final IndicesStatsResponse targetStats = client().admin().indices().prepareStats("target").get();
+ for (final ShardStats shardStats : targetStats.getShards()) {
+ final SeqNoStats seqNoStats = shardStats.getSeqNoStats();
+ final ShardRouting shardRouting = shardStats.getShardRouting();
+ assertThat("failed on " + shardRouting, seqNoStats.getMaxSeqNo(), equalTo(maxSeqNo));
+ assertThat("failed on " + shardRouting, seqNoStats.getLocalCheckpoint(), equalTo(maxSeqNo));
+ assertThat(
+ "failed on " + shardRouting,
+ shardStats.getStats().getSegments().getMaxUnsafeAutoIdTimestamp(),
+ equalTo(maxUnsafeAutoIdTimestamp)
+ );
+ }
+
+ final int size = docs > 0 ? 2 * docs : 1;
+ assertHitCount(client().prepareSearch("target").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+
+ for (int i = docs; i < 2 * docs; i++) {
+ client().prepareIndex("target").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get();
+ }
+ flushAndRefresh();
+ assertHitCount(
+ client().prepareSearch("target").setSize(2 * size).setQuery(new TermsQueryBuilder("foo", "bar")).get(),
+ 2 * docs
+ );
+ assertHitCount(client().prepareSearch("source").setSize(size).setQuery(new TermsQueryBuilder("foo", "bar")).get(), docs);
+ GetSettingsResponse target = client().admin().indices().prepareGetSettings("target").get();
+ assertEquals(version, target.getIndexToSettings().get("target").getAsVersion("index.version.created", null));
+ } finally {
+ // clean up
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(
+ Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), (String) null)
+ )
+ .get();
+ }
+
+ }
+
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/datastream/DataStreamTestCase.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/datastream/DataStreamTestCase.java
index 50ff76c6b62f3..82ab5b0118c0e 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/datastream/DataStreamTestCase.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/datastream/DataStreamTestCase.java
@@ -37,6 +37,7 @@ public AcknowledgedResponse createDataStream(String name) throws Exception {
CreateDataStreamAction.Request request = new CreateDataStreamAction.Request(name);
AcknowledgedResponse response = client().admin().indices().createDataStream(request).get();
assertThat(response.isAcknowledged(), is(true));
+ performRemoteStoreTestAction();
return response;
}
@@ -67,6 +68,7 @@ public RolloverResponse rolloverDataStream(String name) throws Exception {
RolloverResponse response = client().admin().indices().rolloverIndex(request).get();
assertThat(response.isAcknowledged(), is(true));
assertThat(response.isRolledOver(), is(true));
+ performRemoteStoreTestAction();
return response;
}
@@ -109,5 +111,4 @@ public AcknowledgedResponse deleteIndexTemplate(String name) throws Exception {
assertThat(response.isAcknowledged(), is(true));
return response;
}
-
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkProcessorRetryIT.java b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkProcessorRetryIT.java
index 737c0acc309fd..cd6cb0ca3b172 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkProcessorRetryIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkProcessorRetryIT.java
@@ -194,7 +194,7 @@ private static void indexDocs(BulkProcessor processor, int numDocs) {
/**
* Internal helper class to correlate backoff states with bulk responses. This is needed to check whether we maxed out the number
* of retries but still got rejected (which is perfectly fine and can also happen from time to time under heavy load).
- *
+ *
* This implementation relies on an implementation detail in Retry, namely that the bulk listener is notified on the same thread
* as the last call to the backoff policy's iterator. The advantage is that this is non-invasive to the rest of the production code.
*/
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/ingest/AsyncIngestProcessorIT.java b/server/src/internalClusterTest/java/org/opensearch/action/ingest/AsyncIngestProcessorIT.java
index c62c61d5919d6..aefabcb9bc14f 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/ingest/AsyncIngestProcessorIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/ingest/AsyncIngestProcessorIT.java
@@ -69,7 +69,7 @@
/**
* The purpose of this test is to verify that when a processor executes an operation asynchronously that
* the expected result is the same as if the same operation happens synchronously.
- *
+ *
* In this test two test processor are defined that basically do the same operation, but a single processor
* executes asynchronously. The result of the operation should be the same and also the order in which the
* bulk responses are returned should be the same as how the corresponding index requests were defined.
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java
index f0a3b5a5901ce..b1934f901ac65 100644
--- a/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java
@@ -109,8 +109,8 @@ public List getAggregations() {
@Override
public List getFetchSubPhases(FetchPhaseConstructionContext context) {
- /**
- * Set up a fetch sub phase that throws an exception on indices whose name that start with "boom".
+ /*
+ Set up a fetch sub phase that throws an exception on indices whose name that start with "boom".
*/
return Collections.singletonList(fetchContext -> new FetchSubPhaseProcessor() {
@Override
diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/MinimumClusterManagerNodesIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/MinimumClusterManagerNodesIT.java
index 4c8bf24b1655a..84648eda3d38c 100644
--- a/server/src/internalClusterTest/java/org/opensearch/cluster/MinimumClusterManagerNodesIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/cluster/MinimumClusterManagerNodesIT.java
@@ -317,8 +317,8 @@ public void testThreeNodesNoClusterManagerBlock() throws Exception {
);
Settings nonClusterManagerDataPathSettings1 = internalCluster().dataPathSettings(nonClusterManagerNodes.get(0));
Settings nonClusterManagerDataPathSettings2 = internalCluster().dataPathSettings(nonClusterManagerNodes.get(1));
- internalCluster().stopRandomNonClusterManagerNode();
- internalCluster().stopRandomNonClusterManagerNode();
+ internalCluster().stopRandomNodeNotCurrentClusterManager();
+ internalCluster().stopRandomNodeNotCurrentClusterManager();
logger.info("--> verify that there is no cluster-manager anymore on remaining node");
// spin here to wait till the state is set
diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterDisruptionIT.java
index 38b86d307d197..737b272613a44 100644
--- a/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterDisruptionIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterDisruptionIT.java
@@ -461,7 +461,7 @@ public boolean validateClusterForming() {
/**
* Tests that indices are properly deleted even if there is a cluster-manager transition in between.
- * Test for https://github.com/elastic/elasticsearch/issues/11665
+ * Test for Elasticsearch issue #11665
*/
public void testIndicesDeleted() throws Exception {
final String idxName = "test";
diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterManagerDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterManagerDisruptionIT.java
index 1463c45aa9b2f..79f6ba6dfa642 100644
--- a/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterManagerDisruptionIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/discovery/ClusterManagerDisruptionIT.java
@@ -39,6 +39,7 @@
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.coordination.NoClusterManagerBlockService;
import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.cluster.service.ClusterStateStats;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.core.xcontent.MediaTypeRegistry;
@@ -199,6 +200,8 @@ public void testIsolateClusterManagerAndVerifyClusterStateConsensus() throws Exc
}
}
+ ClusterStateStats clusterStateStats = internalCluster().clusterService().getClusterManagerService().getClusterStateStats();
+ assertTrue(clusterStateStats.getUpdateFailed() > 0);
});
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java
index a2864b6dfd1da..70124c8c46700 100644
--- a/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/discovery/DiscoveryDisruptionIT.java
@@ -136,7 +136,7 @@ public void testClusterJoinDespiteOfPublishingIssues() throws Exception {
// shutting down the nodes, to avoid the leakage check tripping
// on the states associated with the commit requests we may have dropped
- internalCluster().stopRandomNonClusterManagerNode();
+ internalCluster().stopRandomNodeNotCurrentClusterManager();
}
public void testClusterFormingWithASlowNode() {
diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java
index 2bab61f3e1c4c..229cd7bffad2f 100644
--- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java
@@ -53,7 +53,7 @@
import org.opensearch.env.NodeEnvironment;
import org.opensearch.index.IndexService;
import org.opensearch.index.IndexSettings;
-import org.opensearch.index.MergePolicyConfig;
+import org.opensearch.index.MergePolicyProvider;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.query.QueryBuilders;
import org.opensearch.index.shard.ShardPath;
@@ -519,7 +519,7 @@ public void testReuseInFileBasedPeerRecovery() throws Exception {
.put("number_of_replicas", 1)
// disable merges to keep segments the same
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
// expire retention leases quickly
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "100ms")
diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java
index 6fcc89cfe9e9a..dfde1b958882c 100644
--- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java
@@ -8,9 +8,12 @@
package org.opensearch.gateway.remote;
+import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest;
+import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.blobstore.BlobPath;
import org.opensearch.common.settings.Settings;
+import org.opensearch.discovery.DiscoveryStats;
import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.repositories.blobstore.BlobStoreRepository;
@@ -19,6 +22,7 @@
import java.nio.charset.StandardCharsets;
import java.util.Base64;
import java.util.Map;
+import java.util.stream.Collectors;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING;
@@ -86,14 +90,105 @@ public void testFullClusterRestoreStaleDelete() throws Exception {
assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size());
- Map indexMetadataMap = remoteClusterStateService.getLatestIndexMetadata(
+ Map indexMetadataMap = remoteClusterStateService.getLatestClusterState(
cluster().getClusterName(),
getClusterState().metadata().clusterUUID()
- );
+ ).getMetadata().getIndices();
assertEquals(0, indexMetadataMap.values().stream().findFirst().get().getNumberOfReplicas());
assertEquals(shardCount, indexMetadataMap.values().stream().findFirst().get().getNumberOfShards());
}
+ public void testRemoteStateStats() {
+ int shardCount = randomIntBetween(1, 2);
+ int replicaCount = 1;
+ int dataNodeCount = shardCount * (replicaCount + 1);
+ int clusterManagerNodeCount = 1;
+ prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount);
+ String clusterManagerNode = internalCluster().getClusterManagerName();
+ String dataNode = internalCluster().getDataNodeNames().stream().collect(Collectors.toList()).get(0);
+
+ // Fetch _nodes/stats
+ NodesStatsResponse nodesStatsResponse = client().admin()
+ .cluster()
+ .prepareNodesStats(clusterManagerNode)
+ .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName())
+ .get();
+
+ // assert cluster state stats
+ assertClusterManagerClusterStateStats(nodesStatsResponse);
+
+ NodesStatsResponse nodesStatsResponseDataNode = client().admin()
+ .cluster()
+ .prepareNodesStats(dataNode)
+ .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName())
+ .get();
+ // assert cluster state stats for data node
+ DiscoveryStats dataNodeDiscoveryStats = nodesStatsResponseDataNode.getNodes().get(0).getDiscoveryStats();
+ assertNotNull(dataNodeDiscoveryStats.getClusterStateStats());
+ assertEquals(0, dataNodeDiscoveryStats.getClusterStateStats().getUpdateSuccess());
+
+ // call nodes/stats with nodeId filter
+ NodesStatsResponse nodesStatsNodeIdFilterResponse = client().admin()
+ .cluster()
+ .prepareNodesStats(dataNode)
+ .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName())
+ .setNodesIds(clusterManagerNode)
+ .get();
+
+ assertClusterManagerClusterStateStats(nodesStatsNodeIdFilterResponse);
+ }
+
+ private void assertClusterManagerClusterStateStats(NodesStatsResponse nodesStatsResponse) {
+ // assert cluster state stats
+ DiscoveryStats discoveryStats = nodesStatsResponse.getNodes().get(0).getDiscoveryStats();
+
+ assertNotNull(discoveryStats.getClusterStateStats());
+ assertTrue(discoveryStats.getClusterStateStats().getUpdateSuccess() > 1);
+ assertEquals(0, discoveryStats.getClusterStateStats().getUpdateFailed());
+ assertTrue(discoveryStats.getClusterStateStats().getUpdateTotalTimeInMillis() > 0);
+ // assert remote state stats
+ assertTrue(discoveryStats.getClusterStateStats().getPersistenceStats().get(0).getSuccessCount() > 1);
+ assertEquals(0, discoveryStats.getClusterStateStats().getPersistenceStats().get(0).getFailedCount());
+ assertTrue(discoveryStats.getClusterStateStats().getPersistenceStats().get(0).getTotalTimeInMillis() > 0);
+ }
+
+ public void testRemoteStateStatsFromAllNodes() {
+ int shardCount = randomIntBetween(1, 5);
+ int replicaCount = 1;
+ int dataNodeCount = shardCount * (replicaCount + 1);
+ int clusterManagerNodeCount = 3;
+ prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount);
+ String[] allNodes = internalCluster().getNodeNames();
+ // call _nodes/stats/discovery from all the nodes
+ for (String node : allNodes) {
+ NodesStatsResponse nodesStatsResponse = client().admin()
+ .cluster()
+ .prepareNodesStats(node)
+ .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName())
+ .get();
+ validateNodesStatsResponse(nodesStatsResponse);
+ }
+
+ // call _nodes/stats/discovery from all the nodes with random nodeId filter
+ for (String node : allNodes) {
+ NodesStatsResponse nodesStatsResponse = client().admin()
+ .cluster()
+ .prepareNodesStats(node)
+ .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName())
+ .setNodesIds(allNodes[randomIntBetween(0, allNodes.length - 1)])
+ .get();
+ validateNodesStatsResponse(nodesStatsResponse);
+ }
+ }
+
+ private void validateNodesStatsResponse(NodesStatsResponse nodesStatsResponse) {
+ // _nodes/stats/discovery must never fail due to any exception
+ assertFalse(nodesStatsResponse.toString().contains("exception"));
+ assertNotNull(nodesStatsResponse.getNodes());
+ assertNotNull(nodesStatsResponse.getNodes().get(0));
+ assertNotNull(nodesStatsResponse.getNodes().get(0).getDiscoveryStats());
+ }
+
private void setReplicaCount(int replicaCount) {
client().admin()
.indices()
diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java
index bb08b19df765b..c394a1f631690 100644
--- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java
@@ -650,7 +650,15 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Engine.DeleteResul
}
}
};
- final IndexShard newShard = newIndexShard(indexService, shard, wrapper, getInstanceFromNode(CircuitBreakerService.class), listener);
+ NodeEnvironment env = getInstanceFromNode(NodeEnvironment.class);
+ final IndexShard newShard = newIndexShard(
+ indexService,
+ shard,
+ wrapper,
+ getInstanceFromNode(CircuitBreakerService.class),
+ env.nodeId(),
+ listener
+ );
shardRef.set(newShard);
recoverShard(newShard);
@@ -674,6 +682,7 @@ public static final IndexShard newIndexShard(
final IndexShard shard,
CheckedFunction wrapper,
final CircuitBreakerService cbs,
+ final String nodeId,
final IndexingOperationListener... listeners
) throws IOException {
ShardRouting initializingShardRouting = getInitializingShardRouting(shard.routingEntry());
@@ -702,7 +711,9 @@ public static final IndexShard newIndexShard(
SegmentReplicationCheckpointPublisher.EMPTY,
null,
null,
- () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL
+ () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL,
+ nodeId,
+ null
);
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java
index f8c2acbf99f70..b431079476624 100644
--- a/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java
@@ -73,7 +73,7 @@
import org.opensearch.env.TestEnvironment;
import org.opensearch.gateway.GatewayMetaState;
import org.opensearch.index.IndexSettings;
-import org.opensearch.index.MergePolicyConfig;
+import org.opensearch.index.MergePolicyProvider;
import org.opensearch.index.MockEngineFactoryPlugin;
import org.opensearch.index.seqno.SeqNoStats;
import org.opensearch.index.translog.TestTranslog;
@@ -135,7 +135,7 @@ public void testCorruptIndex() throws Exception {
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
.put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "-1")
.put(MockEngineSupport.DISABLE_FLUSH_ON_CLOSE.getKey(), true)
.put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), "checksum")
diff --git a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java
index 7e1d0792e3ddb..8291fef5d177b 100644
--- a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java
@@ -72,7 +72,7 @@
import org.opensearch.core.index.shard.ShardId;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.index.IndexSettings;
-import org.opensearch.index.MergePolicyConfig;
+import org.opensearch.index.MergePolicyProvider;
import org.opensearch.index.shard.IndexEventListener;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.IndexShardState;
@@ -167,7 +167,7 @@ public void testCorruptFileAndRecover() throws ExecutionException, InterruptedEx
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, "1")
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "1")
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
// no checkindex - we corrupt shards on purpose
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false)
// no translog based flush - it might change the .liv / segments.N files
@@ -286,7 +286,7 @@ public void testCorruptPrimaryNoReplica() throws ExecutionException, Interrupted
prepareCreate("test").setSettings(
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0")
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) // no checkindex - we corrupt shards on
// purpose
// no translog based flush - it might change the .liv / segments.N files
@@ -552,7 +552,7 @@ public void testCorruptFileThenSnapshotAndRestore() throws ExecutionException, I
prepareCreate("test").setSettings(
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0") // no replicas for this test
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
// no checkindex - we corrupt shards on purpose
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false)
// no translog based flush - it might change the .liv / segments.N files
@@ -624,7 +624,7 @@ public void testReplicaCorruption() throws Exception {
prepareCreate("test").setSettings(
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, cluster().numDataNodes() - 1)
- .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
+ .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) // no checkindex - we corrupt shards on
// purpose
.put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(1, ByteSizeUnit.PB)) // no
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java
index 98a22717019cf..848f6eddbb0df 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java
@@ -539,7 +539,7 @@ public void testCanCache() throws Exception {
assertCacheState(client, "index", 0, 4);
}
- public void testCacheWithFilteredAlias() {
+ public void testCacheWithFilteredAlias() throws InterruptedException {
Client client = client();
Settings settings = Settings.builder()
.put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true)
@@ -562,6 +562,8 @@ public void testCacheWithFilteredAlias() {
OpenSearchAssertions.assertAllSuccessful(forceMergeResponse);
refresh();
+ indexRandomForConcurrentSearch("index");
+
assertCacheState(client, "index", 0, 0);
SearchResponse r1 = client.prepareSearch("index")
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java
index bdefd7a5e199a..f485d4e402b41 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java
@@ -91,7 +91,7 @@ public void testGlobalPrimaryAllocation() throws Exception {
/**
* This test verifies the happy path where primary shard allocation is balanced when multiple indices are created.
- *
+ *
* This test in general passes without primary shard balance as well due to nature of allocation algorithm which
* assigns all primary shards first followed by replica copies.
*/
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java
index 8e68a8bde39d5..1d93eecd6b245 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java
@@ -197,9 +197,10 @@ protected IndexShard getIndexShard(String node, ShardId shardId, String indexNam
protected IndexShard getIndexShard(String node, String indexName) {
final Index index = resolveIndex(indexName);
IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node);
- IndexService indexService = indicesService.indexServiceSafe(index);
+ IndexService indexService = indicesService.indexService(index);
+ assertNotNull(indexService);
final Optional shardId = indexService.shardIds().stream().findFirst();
- return indexService.getShard(shardId.get());
+ return shardId.map(indexService::getShard).orElse(null);
}
protected boolean segmentReplicationWithRemoteEnabled() {
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationDisruptionIT.java
new file mode 100644
index 0000000000000..66b26b5d25cfe
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationDisruptionIT.java
@@ -0,0 +1,167 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.indices.replication;
+
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.opensearch.action.admin.indices.recovery.RecoveryResponse;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.common.bytes.BytesArray;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.recovery.FileChunkRequest;
+import org.opensearch.indices.recovery.RecoveryState;
+import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.test.transport.MockTransportService;
+import org.opensearch.transport.TransportRequest;
+import org.opensearch.transport.TransportService;
+import org.junit.Before;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder;
+
+/**
+ * These tests simulate corruption cases during replication. They are skipped on WindowsFS simulation where file renaming
+ * can fail with an access denied IOException because deletion is not permitted.
+ */
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
+@LuceneTestCase.SuppressFileSystems("WindowsFS")
+public class SegmentReplicationDisruptionIT extends SegmentReplicationBaseIT {
+ @Before
+ private void setup() {
+ internalCluster().startClusterManagerOnlyNode();
+ }
+
+ public void testSendCorruptBytesToReplica() throws Exception {
+ final String primaryNode = internalCluster().startDataOnlyNode();
+ createIndex(
+ INDEX_NAME,
+ Settings.builder()
+ .put(indexSettings())
+ .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+ .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
+ .put("index.refresh_interval", -1)
+ .build()
+ );
+ ensureYellow(INDEX_NAME);
+ final String replicaNode = internalCluster().startDataOnlyNode();
+ ensureGreen(INDEX_NAME);
+
+ MockTransportService primaryTransportService = ((MockTransportService) internalCluster().getInstance(
+ TransportService.class,
+ primaryNode
+ ));
+ CountDownLatch latch = new CountDownLatch(1);
+ AtomicBoolean failed = new AtomicBoolean(false);
+ primaryTransportService.addSendBehavior(
+ internalCluster().getInstance(TransportService.class, replicaNode),
+ (connection, requestId, action, request, options) -> {
+ if (action.equals(SegmentReplicationTargetService.Actions.FILE_CHUNK) && failed.getAndSet(true) == false) {
+ FileChunkRequest req = (FileChunkRequest) request;
+ TransportRequest corrupt = new FileChunkRequest(
+ req.recoveryId(),
+ ((FileChunkRequest) request).requestSeqNo(),
+ ((FileChunkRequest) request).shardId(),
+ ((FileChunkRequest) request).metadata(),
+ ((FileChunkRequest) request).position(),
+ new BytesArray("test"),
+ false,
+ 0,
+ 0L
+ );
+ connection.sendRequest(requestId, action, corrupt, options);
+ latch.countDown();
+ } else {
+ connection.sendRequest(requestId, action, request, options);
+ }
+ }
+ );
+ for (int i = 0; i < 100; i++) {
+ client().prepareIndex(INDEX_NAME)
+ .setId(String.valueOf(i))
+ .setSource(jsonBuilder().startObject().field("field", i).endObject())
+ .get();
+ }
+ final long originalRecoveryTime = getRecoveryStopTime(replicaNode);
+ assertNotEquals(originalRecoveryTime, 0);
+ refresh(INDEX_NAME);
+ latch.await();
+ assertTrue(failed.get());
+ waitForNewPeerRecovery(replicaNode, originalRecoveryTime);
+ // reset checkIndex to ensure our original shard doesn't throw
+ resetCheckIndexStatus();
+ waitForSearchableDocs(100, primaryNode, replicaNode);
+ }
+
+ public void testWipeSegmentBetweenSyncs() throws Exception {
+ internalCluster().startClusterManagerOnlyNode();
+ final String primaryNode = internalCluster().startDataOnlyNode();
+ createIndex(
+ INDEX_NAME,
+ Settings.builder()
+ .put(indexSettings())
+ .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+ .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
+ .put("index.refresh_interval", -1)
+ .build()
+ );
+ ensureYellow(INDEX_NAME);
+ final String replicaNode = internalCluster().startDataOnlyNode();
+ ensureGreen(INDEX_NAME);
+
+ for (int i = 0; i < 10; i++) {
+ client().prepareIndex(INDEX_NAME)
+ .setId(String.valueOf(i))
+ .setSource(jsonBuilder().startObject().field("field", i).endObject())
+ .get();
+ }
+ refresh(INDEX_NAME);
+ ensureGreen(INDEX_NAME);
+ final long originalRecoveryTime = getRecoveryStopTime(replicaNode);
+
+ final IndexShard indexShard = getIndexShard(replicaNode, INDEX_NAME);
+ waitForSearchableDocs(INDEX_NAME, 10, List.of(replicaNode));
+ indexShard.store().directory().deleteFile("_0.si");
+
+ for (int i = 11; i < 21; i++) {
+ client().prepareIndex(INDEX_NAME)
+ .setId(String.valueOf(i))
+ .setSource(jsonBuilder().startObject().field("field", i).endObject())
+ .get();
+ }
+ refresh(INDEX_NAME);
+ waitForNewPeerRecovery(replicaNode, originalRecoveryTime);
+ resetCheckIndexStatus();
+ waitForSearchableDocs(20, primaryNode, replicaNode);
+ }
+
+ private void waitForNewPeerRecovery(String replicaNode, long originalRecoveryTime) throws Exception {
+ assertBusy(() -> {
+ // assert we have a peer recovery after the original
+ final long time = getRecoveryStopTime(replicaNode);
+ assertNotEquals(time, 0);
+ assertNotEquals(originalRecoveryTime, time);
+
+ }, 1, TimeUnit.MINUTES);
+ }
+
+ private long getRecoveryStopTime(String nodeName) {
+ final RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries(INDEX_NAME).get();
+ final List recoveryStates = recoveryResponse.shardRecoveryStates().get(INDEX_NAME);
+ for (RecoveryState recoveryState : recoveryStates) {
+ if (recoveryState.getTargetNode().getName().equals(nodeName)) {
+ return recoveryState.getTimer().stopTime();
+ }
+ }
+ return 0L;
+ }
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java
index 33bc5a8f3afe6..9c93a8f85db8e 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java
@@ -22,6 +22,7 @@
import org.apache.lucene.index.StandardDirectoryReader;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
+import org.opensearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.opensearch.action.admin.indices.alias.Alias;
import org.opensearch.action.admin.indices.flush.FlushRequest;
import org.opensearch.action.admin.indices.stats.IndicesStatsRequest;
@@ -62,6 +63,7 @@
import org.opensearch.core.index.shard.ShardId;
import org.opensearch.core.xcontent.XContentBuilder;
import org.opensearch.index.IndexModule;
+import org.opensearch.index.ReplicationStats;
import org.opensearch.index.SegmentReplicationPerGroupStats;
import org.opensearch.index.SegmentReplicationPressureService;
import org.opensearch.index.SegmentReplicationShardStats;
@@ -94,6 +96,7 @@
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static java.util.Arrays.asList;
@@ -1063,9 +1066,14 @@ public void testScrollCreatedOnReplica() throws Exception {
client(replica).prepareClearScroll().addScrollId(searchResponse.getScrollId()).get();
- currentFiles = List.of(replicaShard.store().directory().listAll());
- assertFalse("Files should be cleaned up post scroll clear request", currentFiles.containsAll(snapshottedSegments));
+ assertBusy(
+ () -> assertFalse(
+ "Files should be cleaned up post scroll clear request",
+ List.of(replicaShard.store().directory().listAll()).containsAll(snapshottedSegments)
+ )
+ );
assertEquals(100, scrollHits);
+
}
/**
@@ -1324,9 +1332,12 @@ public void testPitCreatedOnReplica() throws Exception {
// delete the PIT
DeletePitRequest deletePITRequest = new DeletePitRequest(pitResponse.getId());
client().execute(DeletePitAction.INSTANCE, deletePITRequest).actionGet();
-
- currentFiles = List.of(replicaShard.store().directory().listAll());
- assertFalse("Files should be cleaned up", currentFiles.containsAll(snapshottedSegments));
+ assertBusy(
+ () -> assertFalse(
+ "Files should be cleaned up",
+ List.of(replicaShard.store().directory().listAll()).containsAll(snapshottedSegments)
+ )
+ );
}
/**
@@ -1777,4 +1788,57 @@ public void testRealtimeTermVectorRequestsUnSuccessful() throws IOException {
}
+ public void testReplicaAlreadyAtCheckpoint() throws Exception {
+ final List nodes = new ArrayList<>();
+ final String primaryNode = internalCluster().startDataOnlyNode();
+ nodes.add(primaryNode);
+ final Settings settings = Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build();
+ createIndex(INDEX_NAME, settings);
+ ensureGreen(INDEX_NAME);
+ // start a replica node, initially will be empty with no shard assignment.
+ final String replicaNode = internalCluster().startDataOnlyNode();
+ nodes.add(replicaNode);
+ final String replicaNode2 = internalCluster().startDataOnlyNode();
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareUpdateSettings(INDEX_NAME)
+ .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2))
+ );
+ ensureGreen(INDEX_NAME);
+
+ // index a doc.
+ client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", randomInt()).get();
+ refresh(INDEX_NAME);
+
+ internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
+ ensureYellowAndNoInitializingShards(INDEX_NAME);
+ IndexShard replica_1 = getIndexShard(replicaNode, INDEX_NAME);
+ IndexShard replica_2 = getIndexShard(replicaNode2, INDEX_NAME);
+ // wait until a replica is promoted & finishes engine flip, we don't care which one
+ AtomicReference primary = new AtomicReference<>();
+ assertBusy(() -> {
+ assertTrue("replica should be promoted as a primary", replica_1.routingEntry().primary() || replica_2.routingEntry().primary());
+ primary.set(replica_1.routingEntry().primary() ? replica_1 : replica_2);
+ });
+
+ FlushRequest request = new FlushRequest(INDEX_NAME);
+ request.force(true);
+ primary.get().flush(request);
+
+ assertBusy(() -> {
+ assertEquals(
+ replica_1.getLatestReplicationCheckpoint().getSegmentInfosVersion(),
+ replica_2.getLatestReplicationCheckpoint().getSegmentInfosVersion()
+ );
+ });
+
+ assertBusy(() -> {
+ ClusterStatsResponse clusterStatsResponse = client().admin().cluster().prepareClusterStats().get();
+ ReplicationStats replicationStats = clusterStatsResponse.getIndicesStats().getSegments().getReplicationStats();
+ assertEquals(0L, replicationStats.maxBytesBehind);
+ assertEquals(0L, replicationStats.maxReplicationLag);
+ assertEquals(0L, replicationStats.totalBytesBehind);
+ });
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationRelocationIT.java
index dd832a63d1e66..dbe0b43441f54 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationRelocationIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationRelocationIT.java
@@ -26,6 +26,7 @@
import org.opensearch.index.shard.IndexShard;
import org.opensearch.indices.IndicesService;
import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.test.junit.annotations.TestLogging;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.transport.TransportService;
@@ -55,6 +56,7 @@ private void createIndex(int replicaCount) {
* This test verifies happy path when primary shard is relocated newly added node (target) in the cluster. Before
* relocation and after relocation documents are indexed and documents are verified
*/
+ @TestLogging(reason = "Getting trace logs from replication,shard and allocation package", value = "org.opensearch.indices.replication:TRACE, org.opensearch.index.shard:TRACE, org.opensearch.cluster.routing.allocation:TRACE")
public void testPrimaryRelocation() throws Exception {
final String oldPrimary = internalCluster().startNode();
createIndex(1);
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java
index ae88dd76d54e0..547f9e7a8d380 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java
@@ -509,7 +509,7 @@ public Settings onNodeStopped(String nodeName) throws Exception {
}
/**
- * Test for https://github.com/elastic/elasticsearch/issues/47276 which checks that the persisted metadata on a data node does not
+ * Test for Elasticsearch issue #47276 which checks that the persisted metadata on a data node does not
* become inconsistent when using replicated closed indices.
*/
public void testRelocatedClosedIndexIssue() throws Exception {
diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/stats/IndexStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/stats/IndexStatsIT.java
index a0f01acd1f8e9..9c96d4861d426 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/stats/IndexStatsIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/stats/IndexStatsIT.java
@@ -66,8 +66,8 @@
import org.opensearch.index.IndexModule;
import org.opensearch.index.IndexService;
import org.opensearch.index.IndexSettings;
-import org.opensearch.index.MergePolicyConfig;
import org.opensearch.index.MergeSchedulerConfig;
+import org.opensearch.index.TieredMergePolicyProvider;
import org.opensearch.index.VersionType;
import org.opensearch.index.cache.query.QueryCacheStats;
import org.opensearch.index.engine.VersionConflictEngineException;
@@ -169,7 +169,7 @@ private Settings.Builder settingsBuilder() {
return Settings.builder().put(indexSettings());
}
- public void testFieldDataStats() {
+ public void testFieldDataStats() throws InterruptedException {
assertAcked(
client().admin()
.indices()
@@ -182,6 +182,7 @@ public void testFieldDataStats() {
client().prepareIndex("test").setId("1").setSource("field", "value1", "field2", "value1").execute().actionGet();
client().prepareIndex("test").setId("2").setSource("field", "value2", "field2", "value2").execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet();
+ indexRandomForConcurrentSearch("test");
NodesStatsResponse nodesStats = client().admin().cluster().prepareNodesStats("data:true").setIndices(true).execute().actionGet();
assertThat(
@@ -305,6 +306,7 @@ public void testClearAllCaches() throws Exception {
client().prepareIndex("test").setId("1").setSource("field", "value1").execute().actionGet();
client().prepareIndex("test").setId("2").setSource("field", "value2").execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet();
+ indexRandomForConcurrentSearch("test");
NodesStatsResponse nodesStats = client().admin().cluster().prepareNodesStats("data:true").setIndices(true).execute().actionGet();
assertThat(
@@ -589,8 +591,8 @@ public void testNonThrottleStats() throws Exception {
prepareCreate("test").setSettings(
settingsBuilder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, "1")
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0")
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), "2")
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), "2")
+ .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), "2")
+ .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), "2")
.put(MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING.getKey(), "1")
.put(MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING.getKey(), "10000")
)
@@ -621,8 +623,8 @@ public void testThrottleStats() throws Exception {
prepareCreate("test").setSettings(
settingsBuilder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, "1")
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, "0")
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), "2")
- .put(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), "2")
+ .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), "2")
+ .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), "2")
.put(MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING.getKey(), "1")
.put(MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING.getKey(), "1")
.put(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING.getKey(), Translog.Durability.ASYNC.name())
diff --git a/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java b/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java
index f636185fd4649..d28df90216beb 100644
--- a/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java
@@ -198,11 +198,11 @@ public void testNoRebalanceOnRollingRestart() throws Exception {
// see https://github.com/elastic/elasticsearch/issues/14387
internalCluster().startClusterManagerOnlyNode(Settings.EMPTY);
internalCluster().startDataOnlyNodes(3);
- /**
- * We start 3 nodes and a dedicated cluster-manager. Restart on of the data-nodes and ensure that we got no relocations.
- * Yet we have 6 shards 0 replica so that means if the restarting node comes back both other nodes are subject
- * to relocating to the restarting node since all had 2 shards and now one node has nothing allocated.
- * We have a fix for this to wait until we have allocated unallocated shards now so this shouldn't happen.
+ /*
+ We start 3 nodes and a dedicated cluster-manager. Restart on of the data-nodes and ensure that we got no relocations.
+ Yet we have 6 shards 0 replica so that means if the restarting node comes back both other nodes are subject
+ to relocating to the restarting node since all had 2 shards and now one node has nothing allocated.
+ We have a fix for this to wait until we have allocated unallocated shards now so this shouldn't happen.
*/
prepareCreate("test").setSettings(
Settings.builder()
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java
index bc55f6cc2cbcb..8166c0008ed83 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java
@@ -33,7 +33,6 @@
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT;
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY;
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
public abstract class AbstractRemoteStoreMockRepositoryIntegTestCase extends AbstractSnapshotIntegTestCase {
@@ -107,14 +106,18 @@ public Settings buildRemoteStoreNodeAttributes(Path repoLocation, double ioFailu
.build();
}
- protected void deleteRepo() {
- logger.info("--> Deleting the repository={}", REPOSITORY_NAME);
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
- logger.info("--> Deleting the repository={}", TRANSLOG_REPOSITORY_NAME);
- assertAcked(clusterAdmin().prepareDeleteRepository(TRANSLOG_REPOSITORY_NAME));
+ protected void cleanupRepo() {
+ logger.info("--> Cleanup the repository={}", REPOSITORY_NAME);
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).execute().actionGet();
+ logger.info("--> Cleanup the repository={}", TRANSLOG_REPOSITORY_NAME);
+ clusterAdmin().prepareCleanupRepository(TRANSLOG_REPOSITORY_NAME).execute().actionGet();
}
protected String setup(Path repoLocation, double ioFailureRate, String skipExceptionBlobList, long maxFailure) {
+ return setup(repoLocation, ioFailureRate, skipExceptionBlobList, maxFailure, 0);
+ }
+
+ protected String setup(Path repoLocation, double ioFailureRate, String skipExceptionBlobList, long maxFailure, int replicaCount) {
// The random_control_io_exception_rate setting ensures that 10-25% of all operations to remote store results in
/// IOException. skip_exception_on_verification_file & skip_exception_on_list_blobs settings ensures that the
// repository creation can happen without failure.
@@ -125,8 +128,11 @@ protected String setup(Path repoLocation, double ioFailureRate, String skipExcep
settings.put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT);
}
+ disableRepoConsistencyCheck("Remote Store Creates System Repository");
+
internalCluster().startClusterManagerOnlyNode(settings.build());
String dataNodeName = internalCluster().startDataOnlyNode(settings.build());
+ internalCluster().startDataOnlyNodes(replicaCount, settings.build());
createIndex(INDEX_NAME);
logger.info("--> Created index={}", INDEX_NAME);
ensureYellowAndNoInitializingShards(INDEX_NAME);
@@ -159,7 +165,7 @@ private String getLocalSegmentFilename(String remoteFilename) {
return remoteFilename.split(RemoteSegmentStoreDirectory.SEGMENT_NAME_UUID_SEPARATOR)[0];
}
- private IndexResponse indexSingleDoc() {
+ protected IndexResponse indexSingleDoc() {
return client().prepareIndex(INDEX_NAME)
.setId(UUIDs.randomBase64UUID())
.setSource(randomAlphaOfLength(5), randomAlphaOfLength(5))
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java
index ad3e99dd274ce..99c5d7fb2bae7 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java
@@ -46,7 +46,10 @@ protected Collection> nodePlugins() {
}
protected void restore(String... indices) {
- boolean restoreAllShards = randomBoolean();
+ restore(randomBoolean(), indices);
+ }
+
+ protected void restore(boolean restoreAllShards, String... indices) {
if (restoreAllShards) {
assertAcked(client().admin().indices().prepareClose(indices));
}
@@ -58,7 +61,7 @@ protected void restore(String... indices) {
);
}
- protected void verifyRestoredData(Map indexStats, String indexName) throws Exception {
+ protected void verifyRestoredData(Map indexStats, String indexName, boolean indexMoreData) throws Exception {
ensureYellowAndNoInitializingShards(indexName);
ensureGreen(indexName);
// This is to ensure that shards that were already assigned will get latest count
@@ -68,6 +71,8 @@ protected void verifyRestoredData(Map indexStats, String indexName
30,
TimeUnit.SECONDS
);
+ if (indexMoreData == false) return;
+
IndexResponse response = indexSingleDoc(indexName);
if (indexStats.containsKey(MAX_SEQ_NO_TOTAL + "-shard-" + response.getShardId().id())) {
assertEquals(indexStats.get(MAX_SEQ_NO_TOTAL + "-shard-" + response.getShardId().id()) + 1, response.getSeqNo());
@@ -80,6 +85,10 @@ protected void verifyRestoredData(Map indexStats, String indexName
);
}
+ protected void verifyRestoredData(Map indexStats, String indexName) throws Exception {
+ verifyRestoredData(indexStats, indexName, true);
+ }
+
public void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) {
prepareCluster(numClusterManagerNodes, numDataOnlyNodes, indices, replicaCount, shardCount, Settings.EMPTY);
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexRecoveryIT.java
index 4eb1cc7703735..c957f1b338bfe 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexRecoveryIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteIndexRecoveryIT.java
@@ -23,7 +23,6 @@
import java.nio.file.Path;
import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RemoteIndexRecoveryIT extends IndexRecoveryIT {
@@ -57,7 +56,7 @@ public Settings indexSettings() {
@After
public void teardown() {
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).get();
}
@Override
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java
index 4ebccb9b9e551..21ce4be9981fb 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteRestoreSnapshotIT.java
@@ -11,6 +11,7 @@
import org.opensearch.action.DocWriteResponse;
import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest;
import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.opensearch.action.admin.indices.delete.DeleteIndexRequest;
import org.opensearch.action.admin.indices.get.GetIndexRequest;
import org.opensearch.action.admin.indices.get.GetIndexResponse;
import org.opensearch.action.delete.DeleteResponse;
@@ -20,8 +21,13 @@
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.io.PathUtils;
import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.io.IOUtils;
+import org.opensearch.core.index.Index;
import org.opensearch.core.rest.RestStatus;
+import org.opensearch.index.IndexService;
import org.opensearch.index.IndexSettings;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.IndicesService;
import org.opensearch.indices.replication.common.ReplicationType;
import org.opensearch.snapshots.AbstractSnapshotIntegTestCase;
import org.opensearch.snapshots.SnapshotInfo;
@@ -32,11 +38,15 @@
import org.junit.Before;
import java.io.IOException;
+import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import java.util.Optional;
import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED;
@@ -57,7 +67,7 @@ public void setup() {
@After
public void teardown() {
- assertAcked(clusterAdmin().prepareDeleteRepository(BASE_REMOTE_REPO));
+ clusterAdmin().prepareCleanupRepository(BASE_REMOTE_REPO).get();
}
@Override
@@ -320,6 +330,8 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException {
assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED);
assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED);
ensureGreen(indexName1, restoredIndexName2);
+
+ assertRemoteSegmentsAndTranslogUploaded(restoredIndexName2);
assertDocsPresentInIndex(client, indexName1, numDocsInIndex1);
assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2);
// indexing some new docs and validating
@@ -345,6 +357,97 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException {
assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 4);
}
+ void assertRemoteSegmentsAndTranslogUploaded(String idx) throws IOException {
+ String indexUUID = client().admin().indices().prepareGetSettings(idx).get().getSetting(idx, IndexMetadata.SETTING_INDEX_UUID);
+
+ Path remoteTranslogMetadataPath = Path.of(String.valueOf(remoteRepoPath), indexUUID, "/0/translog/metadata");
+ Path remoteTranslogDataPath = Path.of(String.valueOf(remoteRepoPath), indexUUID, "/0/translog/data");
+ Path segmentMetadataPath = Path.of(String.valueOf(remoteRepoPath), indexUUID, "/0/segments/metadata");
+ Path segmentDataPath = Path.of(String.valueOf(remoteRepoPath), indexUUID, "/0/segments/data");
+
+ try (
+ Stream translogMetadata = Files.list(remoteTranslogMetadataPath);
+ Stream translogData = Files.list(remoteTranslogDataPath);
+ Stream segmentMetadata = Files.list(segmentMetadataPath);
+ Stream segmentData = Files.list(segmentDataPath);
+
+ ) {
+ assertTrue(translogData.count() > 0);
+ assertTrue(translogMetadata.count() > 0);
+ assertTrue(segmentMetadata.count() > 0);
+ assertTrue(segmentData.count() > 0);
+ }
+
+ }
+
+ public void testRemoteRestoreIndexRestoredFromSnapshot() throws IOException, ExecutionException, InterruptedException {
+ internalCluster().startClusterManagerOnlyNode();
+ internalCluster().startDataOnlyNodes(2);
+
+ String indexName1 = "testindex1";
+ String snapshotRepoName = "test-restore-snapshot-repo";
+ String snapshotName1 = "test-restore-snapshot1";
+ Path absolutePath1 = randomRepoPath().toAbsolutePath();
+ logger.info("Snapshot Path [{}]", absolutePath1);
+
+ createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true));
+
+ Settings indexSettings = getIndexSettings(1, 0).build();
+ createIndex(indexName1, indexSettings);
+
+ final int numDocsInIndex1 = randomIntBetween(20, 30);
+ indexDocuments(client(), indexName1, numDocsInIndex1);
+ flushAndRefresh(indexName1);
+ ensureGreen(indexName1);
+
+ logger.info("--> snapshot");
+ SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(Arrays.asList(indexName1)));
+ assertThat(snapshotInfo1.successfulShards(), greaterThan(0));
+ assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards()));
+ assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS));
+
+ assertAcked(client().admin().indices().delete(new DeleteIndexRequest(indexName1)).get());
+ assertFalse(indexExists(indexName1));
+
+ RestoreSnapshotResponse restoreSnapshotResponse1 = client().admin()
+ .cluster()
+ .prepareRestoreSnapshot(snapshotRepoName, snapshotName1)
+ .setWaitForCompletion(false)
+ .setIndices(indexName1)
+ .get();
+
+ assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED);
+ ensureGreen(indexName1);
+ assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1);
+
+ assertRemoteSegmentsAndTranslogUploaded(indexName1);
+
+ // Clear the local data before stopping the node. This will make sure that remote translog is empty.
+ IndexShard indexShard = getIndexShard(primaryNodeName(indexName1), indexName1);
+ try (Stream files = Files.list(indexShard.shardPath().resolveTranslog())) {
+ IOUtils.deleteFilesIgnoringExceptions(files.collect(Collectors.toList()));
+ }
+ internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(indexName1)));
+
+ ensureRed(indexName1);
+
+ client().admin()
+ .cluster()
+ .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1).restoreAllShards(false), PlainActionFuture.newFuture());
+
+ ensureGreen(indexName1);
+ assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1);
+ }
+
+ protected IndexShard getIndexShard(String node, String indexName) {
+ final Index index = resolveIndex(indexName);
+ IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node);
+ IndexService indexService = indicesService.indexService(index);
+ assertNotNull(indexService);
+ final Optional shardId = indexService.shardIds().stream().findFirst();
+ return shardId.map(indexService::getShard).orElse(null);
+ }
+
public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException {
String clusterManagerNode = internalCluster().startClusterManagerOnlyNode();
String primary = internalCluster().startDataOnlyNode();
@@ -422,7 +525,7 @@ public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException
assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2);
}
- public void testRestoreShallowSnapshotRepositoryOverriden() throws ExecutionException, InterruptedException {
+ public void testRestoreShallowSnapshotRepository() throws ExecutionException, InterruptedException {
String indexName1 = "testindex1";
String snapshotRepoName = "test-restore-snapshot-repo";
String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX;
@@ -464,22 +567,74 @@ public void testRestoreShallowSnapshotRepositoryOverriden() throws ExecutionExce
assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards()));
assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS));
- createRepository(BASE_REMOTE_REPO, "fs", absolutePath2);
-
- RestoreSnapshotResponse restoreSnapshotResponse = client.admin()
+ client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get();
+ createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath);
+ RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin()
.cluster()
.prepareRestoreSnapshot(snapshotRepoName, snapshotName1)
.setWaitForCompletion(true)
.setIndices(indexName1)
.setRenamePattern(indexName1)
.setRenameReplacement(restoredIndexName1)
+ .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated)
.get();
- assertTrue(restoreSnapshotResponse.getRestoreInfo().failedShards() > 0);
+ assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0);
+ ensureGreen(restoredIndexName1);
+ assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1);
- ensureRed(restoredIndexName1);
+ // indexing some new docs and validating
+ indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2);
+ ensureGreen(restoredIndexName1);
+ assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2);
+ }
+
+ public void testRestoreShallowSnapshotIndexAfterSnapshot() throws ExecutionException, InterruptedException {
+ String indexName1 = "testindex1";
+ String snapshotRepoName = "test-restore-snapshot-repo";
+ String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX;
+ String snapshotName1 = "test-restore-snapshot1";
+ Path absolutePath1 = randomRepoPath().toAbsolutePath();
+ Path absolutePath2 = randomRepoPath().toAbsolutePath();
+ String[] pathTokens = absolutePath1.toString().split("/");
+ String basePath = pathTokens[pathTokens.length - 1];
+ Arrays.copyOf(pathTokens, pathTokens.length - 1);
+ Path location = PathUtils.get(String.join("/", pathTokens));
+ pathTokens = absolutePath2.toString().split("/");
+ String basePath2 = pathTokens[pathTokens.length - 1];
+ Arrays.copyOf(pathTokens, pathTokens.length - 1);
+ Path location2 = PathUtils.get(String.join("/", pathTokens));
+ logger.info("Path 1 [{}]", absolutePath1);
+ logger.info("Path 2 [{}]", absolutePath2);
+ String restoredIndexName1 = indexName1 + "-restored";
+
+ createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true));
+
+ Client client = client();
+ Settings indexSettings = Settings.builder()
+ .put(super.indexSettings())
+ .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT)
+ .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+ .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
+ .build();
+ createIndex(indexName1, indexSettings);
+
+ int numDocsInIndex1 = randomIntBetween(2, 5);
+ indexDocuments(client, indexName1, numDocsInIndex1);
+
+ ensureGreen(indexName1);
+
+ logger.info("--> snapshot");
+ SnapshotInfo snapshotInfo1 = createSnapshot(snapshotRepoName, snapshotName1, new ArrayList<>(List.of(indexName1)));
+ assertThat(snapshotInfo1.successfulShards(), greaterThan(0));
+ assertThat(snapshotInfo1.successfulShards(), equalTo(snapshotInfo1.totalShards()));
+ assertThat(snapshotInfo1.state(), equalTo(SnapshotState.SUCCESS));
+
+ int extraNumDocsInIndex1 = randomIntBetween(20, 50);
+ indexDocuments(client, indexName1, extraNumDocsInIndex1);
+ refresh(indexName1);
- client().admin().indices().close(Requests.closeIndexRequest(restoredIndexName1)).get();
+ client().admin().indices().close(Requests.closeIndexRequest(indexName1)).get();
createRepository(remoteStoreRepoNameUpdated, "fs", remoteRepoPath);
RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin()
.cluster()
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java
similarity index 57%
rename from server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java
rename to server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java
index d02c5bf54fbed..f19c9db7874db 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java
@@ -11,13 +11,20 @@
import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStats;
import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsResponse;
import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
+import org.opensearch.action.admin.indices.flush.FlushResponse;
import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.common.util.concurrent.AbstractAsyncTask;
+import org.opensearch.common.util.concurrent.UncategorizedExecutionException;
import org.opensearch.core.common.bytes.BytesArray;
import org.opensearch.core.common.bytes.BytesReference;
import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException;
import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.index.IndexService;
import org.opensearch.index.remote.RemoteSegmentTransferTracker;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.IndicesService;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.snapshots.mockstore.MockRepository;
import org.opensearch.test.OpenSearchIntegTestCase;
@@ -33,7 +40,7 @@
import static org.opensearch.index.remote.RemoteStorePressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
-public class RemoteStoreBackpressureIT extends AbstractRemoteStoreMockRepositoryIntegTestCase {
+public class RemoteStoreBackpressureAndResiliencyIT extends AbstractRemoteStoreMockRepositoryIntegTestCase {
public void testWritesRejectedDueToConsecutiveFailureBreach() throws Exception {
// Here the doc size of the request remains same throughout the test. After initial indexing, all remote store interactions
// fail leading to consecutive failure limit getting exceeded and leading to rejections.
@@ -49,7 +56,7 @@ public void testWritesRejectedDueToBytesLagBreach() throws Exception {
public void testWritesRejectedDueToTimeLagBreach() throws Exception {
// Initially indexing happens with doc size of 1KB, then all remote store interactions start failing. Now, the
// indexing happens with doc size of 1 byte leading to time lag limit getting exceeded and leading to rejections.
- validateBackpressure(ByteSizeUnit.KB.toIntBytes(1), 20, ByteSizeUnit.BYTES.toIntBytes(1), 15, "time_lag");
+ validateBackpressure(ByteSizeUnit.KB.toIntBytes(1), 20, ByteSizeUnit.BYTES.toIntBytes(1), 3, "time_lag");
}
private void validateBackpressure(
@@ -112,7 +119,7 @@ private void validateBackpressure(
stats = stats();
indexDocAndRefresh(initialSource, initialDocsToIndex);
assertEquals(rejectionCount, stats.rejectionCount);
- deleteRepo();
+ cleanupRepo();
}
private RemoteSegmentTransferTracker.Stats stats() {
@@ -126,11 +133,13 @@ private RemoteSegmentTransferTracker.Stats stats() {
return matches.get(0).getSegmentStats();
}
- private void indexDocAndRefresh(BytesReference source, int iterations) {
+ private void indexDocAndRefresh(BytesReference source, int iterations) throws InterruptedException {
for (int i = 0; i < iterations; i++) {
client().prepareIndex(INDEX_NAME).setSource(source, MediaTypeRegistry.JSON).get();
refresh(INDEX_NAME);
}
+ Thread.sleep(250);
+ client().prepareIndex(INDEX_NAME).setSource(source, MediaTypeRegistry.JSON).get();
}
/**
@@ -156,4 +165,98 @@ private String generateString(int sizeInBytes) {
sb.append("}");
return sb.toString();
}
+
+ /**
+ * Fixes Github#10398
+ */
+ public void testAsyncTrimTaskSucceeds() {
+ Path location = randomRepoPath().toAbsolutePath();
+ String dataNodeName = setup(location, 0d, "metadata", Long.MAX_VALUE);
+
+ logger.info("Increasing the frequency of async trim task to ensure it runs in background while indexing");
+ IndexService indexService = internalCluster().getInstance(IndicesService.class, dataNodeName).iterator().next();
+ ((AbstractAsyncTask) indexService.getTrimTranslogTask()).setInterval(TimeValue.timeValueMillis(100));
+
+ logger.info("--> Indexing data");
+ indexData(randomIntBetween(2, 5), true);
+ logger.info("--> Indexing succeeded");
+
+ MockRepository translogRepo = (MockRepository) internalCluster().getInstance(RepositoriesService.class, dataNodeName)
+ .repository(TRANSLOG_REPOSITORY_NAME);
+ logger.info("--> Failing all remote store interaction");
+ translogRepo.setRandomControlIOExceptionRate(1d);
+
+ for (int i = 0; i < randomIntBetween(5, 10); i++) {
+ UncategorizedExecutionException exception = assertThrows(UncategorizedExecutionException.class, this::indexSingleDoc);
+ assertEquals("Failed execution", exception.getMessage());
+ }
+
+ translogRepo.setRandomControlIOExceptionRate(0d);
+ indexSingleDoc();
+ logger.info("Indexed single doc successfully");
+ }
+
+ /**
+ * Fixes Github#10400
+ */
+ public void testSkipLoadGlobalCheckpointToReplicationTracker() {
+ Path location = randomRepoPath().toAbsolutePath();
+ String dataNodeName = setup(location, 0d, "metadata", Long.MAX_VALUE);
+
+ logger.info("--> Indexing data");
+ indexData(randomIntBetween(1, 2), true);
+ logger.info("--> Indexing succeeded");
+
+ IndexService indexService = internalCluster().getInstance(IndicesService.class, dataNodeName).iterator().next();
+ IndexShard indexShard = indexService.getShard(0);
+ indexShard.failShard("failing shard", null);
+
+ ensureRed(INDEX_NAME);
+
+ MockRepository translogRepo = (MockRepository) internalCluster().getInstance(RepositoriesService.class, dataNodeName)
+ .repository(TRANSLOG_REPOSITORY_NAME);
+ logger.info("--> Failing all remote store interaction");
+ translogRepo.setRandomControlIOExceptionRate(1d);
+ client().admin().cluster().prepareReroute().setRetryFailed(true).get();
+ // CLuster stays red still as the remote interactions are still failing
+ ensureRed(INDEX_NAME);
+
+ logger.info("Retrying to allocate failed shards");
+ client().admin().cluster().prepareReroute().setRetryFailed(true).get();
+ // CLuster stays red still as the remote interactions are still failing
+ ensureRed(INDEX_NAME);
+
+ logger.info("Stop failing all remote store interactions");
+ translogRepo.setRandomControlIOExceptionRate(0d);
+ client().admin().cluster().prepareReroute().setRetryFailed(true).get();
+ ensureGreen(INDEX_NAME);
+ }
+
+ public void testFlushDuringRemoteUploadFailures() {
+ Path location = randomRepoPath().toAbsolutePath();
+ String dataNodeName = setup(location, 0d, "metadata", Long.MAX_VALUE);
+
+ logger.info("--> Indexing data");
+ indexData(randomIntBetween(1, 2), true);
+ logger.info("--> Indexing succeeded");
+ ensureGreen(INDEX_NAME);
+
+ MockRepository translogRepo = (MockRepository) internalCluster().getInstance(RepositoriesService.class, dataNodeName)
+ .repository(TRANSLOG_REPOSITORY_NAME);
+ logger.info("--> Failing all remote store interaction");
+ translogRepo.setRandomControlIOExceptionRate(1d);
+
+ Exception ex = assertThrows(UncategorizedExecutionException.class, () -> indexSingleDoc());
+ assertEquals("Failed execution", ex.getMessage());
+
+ FlushResponse flushResponse = client().admin().indices().prepareFlush(INDEX_NAME).setForce(true).execute().actionGet();
+ assertEquals(1, flushResponse.getFailedShards());
+ ensureGreen(INDEX_NAME);
+
+ logger.info("--> Stop failing all remote store interactions");
+ translogRepo.setRandomControlIOExceptionRate(0d);
+ flushResponse = client().admin().indices().prepareFlush(INDEX_NAME).setForce(true).execute().actionGet();
+ assertEquals(1, flushResponse.getSuccessfulShards());
+ assertEquals(0, flushResponse.getFailedShards());
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java
index 157f8e41fee24..8b4981a15433a 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java
@@ -27,6 +27,7 @@
import org.opensearch.index.IndexSettings;
import org.opensearch.index.mapper.MapperService;
import org.opensearch.indices.replication.common.ReplicationType;
+import org.opensearch.repositories.RepositoriesService;
import org.opensearch.repositories.blobstore.BlobStoreRepository;
import org.opensearch.repositories.fs.FsRepository;
import org.opensearch.test.OpenSearchIntegTestCase;
@@ -50,13 +51,12 @@
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT;
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY;
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
public class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase {
protected static final String REPOSITORY_NAME = "test-remote-store-repo";
protected static final String REPOSITORY_2_NAME = "test-remote-store-repo-2";
protected static final int SHARD_COUNT = 1;
- protected static final int REPLICA_COUNT = 1;
+ protected static int REPLICA_COUNT = 1;
protected static final String TOTAL_OPERATIONS = "total-operations";
protected static final String REFRESHED_OR_FLUSHED_OPERATIONS = "refreshed-or-flushed-operations";
protected static final String MAX_SEQ_NO_TOTAL = "max-seq-no-total";
@@ -271,7 +271,6 @@ public static Settings buildRemoteStoreNodeAttributes(
if (withRateLimiterAttributes) {
settings.put(segmentRepoSettingsAttributeKeyPrefix + "compress", randomBoolean())
- .put(segmentRepoSettingsAttributeKeyPrefix + "max_remote_download_bytes_per_sec", "4kb")
.put(segmentRepoSettingsAttributeKeyPrefix + "chunk_size", 200, ByteSizeUnit.BYTES);
}
@@ -314,8 +313,8 @@ public void teardown() {
clusterSettingsSuppliedByTest = false;
assertRemoteStoreRepositoryOnAllNodes(REPOSITORY_NAME);
assertRemoteStoreRepositoryOnAllNodes(REPOSITORY_2_NAME);
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_2_NAME));
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).get();
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_2_NAME).get();
}
public RepositoryMetadata buildRepositoryMetadata(DiscoveryNode node, String name) {
@@ -343,11 +342,24 @@ public void assertRemoteStoreRepositoryOnAllNodes(String repositoryName) {
.custom(RepositoriesMetadata.TYPE);
RepositoryMetadata actualRepository = repositories.repository(repositoryName);
+ final RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class);
+ final BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(repositoryName);
+
for (String nodeName : internalCluster().getNodeNames()) {
ClusterService clusterService = internalCluster().getInstance(ClusterService.class, nodeName);
DiscoveryNode node = clusterService.localNode();
RepositoryMetadata expectedRepository = buildRepositoryMetadata(node, repositoryName);
- assertTrue(actualRepository.equalsIgnoreGenerations(expectedRepository));
+
+ // Validated that all the restricted settings are entact on all the nodes.
+ repository.getRestrictedSystemRepositorySettings()
+ .stream()
+ .forEach(
+ setting -> assertEquals(
+ String.format(Locale.ROOT, "Restricted Settings mismatch [%s]", setting.getKey()),
+ setting.get(actualRepository.settings()),
+ setting.get(expectedRepository.settings())
+ )
+ );
}
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java
index 5e92bb195680b..c61e2ec6e4f6c 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java
@@ -8,23 +8,41 @@
package org.opensearch.remotestore;
-import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreResponse;
import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
-import org.opensearch.action.support.PlainActionFuture;
+import org.opensearch.action.admin.indices.datastream.DataStreamRolloverIT;
+import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest;
+import org.opensearch.action.admin.indices.template.put.PutIndexTemplateRequest;
+import org.opensearch.cluster.ClusterState;
+import org.opensearch.cluster.block.ClusterBlockException;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.cluster.metadata.IndexTemplateMetadata;
+import org.opensearch.cluster.metadata.Metadata;
+import org.opensearch.cluster.metadata.RepositoriesMetadata;
import org.opensearch.common.settings.Settings;
+import org.opensearch.gateway.remote.ClusterMetadataManifest;
+import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata;
import org.opensearch.gateway.remote.RemoteClusterStateService;
+import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
import java.io.IOException;
import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ExecutionException;
+import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING;
+import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_READ_ONLY_SETTING;
+import static org.opensearch.cluster.metadata.Metadata.CLUSTER_READ_ONLY_BLOCK;
+import static org.opensearch.cluster.metadata.Metadata.SETTING_READ_ONLY_SETTING;
import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING;
import static org.opensearch.indices.ShardLimitValidator.SETTING_CLUSTER_MAX_SHARDS_PER_NODE;
-import static org.opensearch.indices.ShardLimitValidator.SETTING_MAX_SHARDS_PER_CLUSTER_KEY;
+import static org.opensearch.repositories.blobstore.BlobStoreRepository.SYSTEM_REPOSITORY_SETTING;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RemoteStoreClusterStateRestoreIT extends BaseRemoteStoreRestoreIT {
@@ -48,68 +66,122 @@ private Map initialTestSetup(int shardCount, int replicaCount, int
private void resetCluster(int dataNodeCount, int clusterManagerNodeCount) {
internalCluster().stopAllNodes();
- addNewNodes(dataNodeCount, clusterManagerNodeCount);
+ internalCluster().startClusterManagerOnlyNodes(clusterManagerNodeCount);
+ internalCluster().startDataOnlyNodes(dataNodeCount);
}
- private void restoreAndValidate(String clusterUUID, Map indexStats) throws Exception {
- restoreAndValidate(clusterUUID, indexStats, true);
+ protected void verifyRedIndicesAndTriggerRestore(Map indexStats, String indexName, boolean indexMoreDocs)
+ throws Exception {
+ ensureRed(indexName);
+ restore(false, indexName);
+ verifyRestoredData(indexStats, indexName, indexMoreDocs);
}
- private void restoreAndValidate(String clusterUUID, Map indexStats, boolean validate) throws Exception {
- // TODO once auto restore is merged, the remote cluster state will be restored
+ public void testFullClusterRestore() throws Exception {
+ int shardCount = randomIntBetween(1, 2);
+ int replicaCount = 1;
+ int dataNodeCount = shardCount * (replicaCount + 1);
+ int clusterManagerNodeCount = 1;
- if (validate) {
- // Step - 4 validation restore is successful.
- ensureGreen(INDEX_NAME);
- verifyRestoredData(indexStats, INDEX_NAME);
- }
- }
+ // Step - 1 index some data to generate files in remote directory
+ Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, 1);
+ String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ long prevClusterStateVersion = clusterService().state().version();
- private void restoreAndValidateFails(
- String clusterUUID,
- PlainActionFuture actionListener,
- Class extends Throwable> clazz,
- String errorSubString
- ) {
+ // Step - 2 Replace all nodes in the cluster with new nodes. This ensures new cluster state doesn't have previous index metadata
+ resetCluster(dataNodeCount, clusterManagerNodeCount);
+
+ String newClusterUUID = clusterService().state().metadata().clusterUUID();
+ assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
+
+ // Step - 3 validate cluster state restored
+ long newClusterStateVersion = clusterService().state().version();
+ assert prevClusterStateVersion < newClusterStateVersion : String.format(
+ Locale.ROOT,
+ "ClusterState version is not restored. previousClusterVersion: [%s] is greater than current [%s]",
+ prevClusterStateVersion,
+ newClusterStateVersion
+ );
+ validateMetadata(List.of(INDEX_NAME));
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, true);
- try {
- restoreAndValidate(clusterUUID, null, false);
- } catch (Exception e) {
- assertTrue(
- String.format(Locale.ROOT, "%s %s", clazz, e),
- clazz.isAssignableFrom(e.getClass())
- || clazz.isAssignableFrom(e.getCause().getClass())
- || (e.getCause().getCause() != null && clazz.isAssignableFrom(e.getCause().getCause().getClass()))
- );
- assertTrue(
- String.format(Locale.ROOT, "Error message mismatch. Expected: [%s]. Actual: [%s]", errorSubString, e.getMessage()),
- e.getMessage().contains(errorSubString)
- );
- }
}
- @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9834")
- public void testFullClusterRestore() throws Exception {
+ /**
+ * This test scenario covers the case where right after remote state restore and persisting it to disk via LucenePersistedState, full cluster restarts.
+ * This is a special case for remote state as at this point cluster uuid in the restored state is still ClusterState.UNKNOWN_UUID as we persist it disk.
+ * After restart the local disk state will be read but should be again overridden with remote state.
+ *
+ * 1. Form a cluster and index few docs
+ * 2. Replace all nodes to remove all local disk state
+ * 3. Start cluster manager node without correct seeding to ensure local disk state is written with cluster uuid ClusterState.UNKNOWN_UUID but with remote restored Metadata
+ * 4. Restart the cluster manager node with correct seeding.
+ * 5. After restart the cluster manager picks up the local disk state with has same Metadata as remote but cluster uuid is still ClusterState.UNKNOWN_UUID
+ * 6. The cluster manager will try to restore from remote again.
+ * 7. Metadata loaded from local disk state will be overridden with remote Metadata and no conflict should arise.
+ * 8. Add data nodes to recover index data
+ * 9. Verify Metadata and index data is restored.
+ */
+ public void testFullClusterRestoreDoesntFailWithConflictingLocalState() throws Exception {
int shardCount = randomIntBetween(1, 2);
int replicaCount = 1;
int dataNodeCount = shardCount * (replicaCount + 1);
int clusterManagerNodeCount = 1;
- // Step - 1 index some data to generate files in remote directory
+ // index some data to generate files in remote directory
Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, 1);
String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ long prevClusterStateVersion = clusterService().state().version();
- // Step - 2 Replace all nodes in the cluster with new nodes. This ensures new cluster state doesn't have previous index metadata
- resetCluster(dataNodeCount, clusterManagerNodeCount);
+ // stop all nodes
+ internalCluster().stopAllNodes();
+
+ // start a cluster manager node with no cluster manager seeding.
+ // This should fail with IllegalStateException as cluster manager fails to form without any initial seed
+ assertThrows(
+ IllegalStateException.class,
+ () -> internalCluster().startClusterManagerOnlyNodes(
+ clusterManagerNodeCount,
+ Settings.builder()
+ .putList(INITIAL_CLUSTER_MANAGER_NODES_SETTING.getKey()) // disable seeding during bootstrapping
+ .build()
+ )
+ );
+ // verify cluster manager not elected
String newClusterUUID = clusterService().state().metadata().clusterUUID();
+ assert Objects.equals(newClusterUUID, ClusterState.UNKNOWN_UUID)
+ : "Disabling Cluster manager seeding failed. cluster uuid is not unknown";
+
+ // restart cluster manager with correct seed
+ internalCluster().fullRestart(new InternalTestCluster.RestartCallback() {
+ @Override
+ public Settings onNodeStopped(String nodeName) {
+ return Settings.builder()
+ .putList(INITIAL_CLUSTER_MANAGER_NODES_SETTING.getKey(), nodeName) // Seed with correct Cluster Manager node
+ .build();
+ }
+ });
+
+ // validate new cluster state formed
+ newClusterUUID = clusterService().state().metadata().clusterUUID();
+ assert !Objects.equals(newClusterUUID, ClusterState.UNKNOWN_UUID) : "cluster restart not successful. cluster uuid is still unknown";
assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
- // Step - 3 Trigger full cluster restore and validate
- restoreAndValidate(prevClusterUUID, indexStats);
+ long newClusterStateVersion = clusterService().state().version();
+ assert prevClusterStateVersion < newClusterStateVersion : String.format(
+ Locale.ROOT,
+ "ClusterState version is not restored. previousClusterVersion: [%s] is greater than current [%s]",
+ prevClusterStateVersion,
+ newClusterStateVersion
+ );
+ validateMetadata(List.of(INDEX_NAME));
+
+ // start data nodes to trigger index data recovery
+ internalCluster().startDataOnlyNodes(dataNodeCount);
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, true);
}
- @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9834")
public void testFullClusterRestoreMultipleIndices() throws Exception {
int shardCount = randomIntBetween(1, 2);
int replicaCount = 1;
@@ -124,8 +196,10 @@ public void testFullClusterRestoreMultipleIndices() throws Exception {
Map indexStats2 = indexData(1, false, secondIndexName);
assertEquals((shardCount + 1) * (replicaCount + 1), getNumShards(secondIndexName).totalNumShards);
ensureGreen(secondIndexName);
+ updateIndexBlock(true, secondIndexName);
String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ long prevClusterStateVersion = clusterService().state().version();
// Step - 2 Replace all nodes in the cluster with new nodes. This ensures new cluster state doesn't have previous index metadata
resetCluster(dataNodeCount, clusterManagerNodeCount);
@@ -133,99 +207,157 @@ public void testFullClusterRestoreMultipleIndices() throws Exception {
String newClusterUUID = clusterService().state().metadata().clusterUUID();
assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
- // Step - 3 Trigger full cluster restore
- restoreAndValidate(prevClusterUUID, indexStats);
- ensureGreen(secondIndexName);
- verifyRestoredData(indexStats2, secondIndexName);
+ // Step - 3 validate cluster state restored
+ long newClusterStateVersion = clusterService().state().version();
+ assert prevClusterStateVersion < newClusterStateVersion : String.format(
+ Locale.ROOT,
+ "ClusterState version is not restored. previousClusterVersion: [%s] is greater than current [%s]",
+ prevClusterStateVersion,
+ newClusterStateVersion
+ );
+ validateMetadata(List.of(INDEX_NAME, secondIndexName));
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, false);
+ verifyRedIndicesAndTriggerRestore(indexStats2, secondIndexName, false);
+ assertTrue(INDEX_READ_ONLY_SETTING.get(clusterService().state().metadata().index(secondIndexName).getSettings()));
+ assertThrows(ClusterBlockException.class, () -> indexSingleDoc(secondIndexName));
+ // Test is complete
+
+ // Remove the block to ensure proper cleanup
+ updateIndexBlock(false, secondIndexName);
}
- @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9834")
- public void testFullClusterRestoreFailureValidationFailures() throws Exception {
+ public void testFullClusterRestoreManifestFilePointsToInvalidIndexMetadataPathThrowsException() throws Exception {
int shardCount = randomIntBetween(1, 2);
int replicaCount = 1;
int dataNodeCount = shardCount * (replicaCount + 1);
int clusterManagerNodeCount = 1;
- // index some data to generate files in remote directory
- Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount);
- String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ // Step - 1 index some data to generate files in remote directory
+ initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount);
- // Start of Test - 1
- // Test - 1 Trigger full cluster restore and validate it fails due to incorrect cluster UUID
- PlainActionFuture future = PlainActionFuture.newFuture();
- restoreAndValidateFails("randomUUID", future, IllegalStateException.class, "Remote Cluster State not found - randomUUID");
- // End of Test - 1
-
- // Start of Test - 3
- // Test - 2 Trigger full cluster restore and validate it fails due to cluster UUID same as current cluster UUID
- future = PlainActionFuture.newFuture();
- restoreAndValidateFails(
- clusterService().state().metadata().clusterUUID(),
- future,
- IllegalArgumentException.class,
- "clusterUUID to restore from should be different from current cluster UUID"
- );
- // End of Test - 2
+ String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ String clusterName = clusterService().state().getClusterName().value();
- // Start of Test - 3
// Step - 2 Replace all nodes in the cluster with new nodes. This ensures new cluster state doesn't have previous index metadata
- // Restarting cluster with just 1 data node helps with applying cluster settings
- resetCluster(1, clusterManagerNodeCount);
- String newClusterUUID = clusterService().state().metadata().clusterUUID();
- assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
-
- reduceShardLimits(1, 1);
-
- // Step - 4 Trigger full cluster restore and validate it fails
- future = PlainActionFuture.newFuture();
- restoreAndValidateFails(
- prevClusterUUID,
- future,
- IllegalArgumentException.class,
- "this action would add [2] total shards, but this cluster currently has [0]/[1] maximum shards open"
- );
- resetShardLimits();
- // End of Test - 3
+ internalCluster().stopAllNodes();
+ // Step - 3 Delete index metadata file in remote
+ try {
+ Files.move(
+ segmentRepoPath.resolve(
+ RemoteClusterStateService.encodeString(clusterName) + "/cluster-state/" + prevClusterUUID + "/index"
+ ),
+ segmentRepoPath.resolve("cluster-state/")
+ );
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ assertThrows(IllegalStateException.class, () -> addNewNodes(dataNodeCount, clusterManagerNodeCount));
+ // Test is complete
- // Start of Test - 4
- // Test -4 Reset cluster and trigger full restore with same name index in the cluster
- // Test -4 Add required nodes for this test after last reset.
- addNewNodes(dataNodeCount - 1, 0);
+ // Starting a node without remote state to ensure test cleanup
+ internalCluster().startNode(Settings.builder().put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), false).build());
+ }
- newClusterUUID = clusterService().state().metadata().clusterUUID();
- assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
+ public void testRemoteStateFullRestart() throws Exception {
+ int shardCount = randomIntBetween(1, 2);
+ int replicaCount = 1;
+ int dataNodeCount = shardCount * (replicaCount + 1);
+ int clusterManagerNodeCount = 3;
- // Test -4 Step - 2 Create a new index with same name
- createIndex(INDEX_NAME, remoteStoreIndexSettings(0, 1));
- ensureYellowAndNoInitializingShards(INDEX_NAME);
+ Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount);
+ String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ long prevClusterStateVersion = clusterService().state().version();
+ // Delete index metadata file in remote
+ try {
+ Files.move(
+ segmentRepoPath.resolve(
+ RemoteClusterStateService.encodeString(clusterService().state().getClusterName().value())
+ + "/cluster-state/"
+ + prevClusterUUID
+ + "/manifest"
+ ),
+ segmentRepoPath.resolve("cluster-state/")
+ );
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ internalCluster().fullRestart();
ensureGreen(INDEX_NAME);
+ String newClusterUUID = clusterService().state().metadata().clusterUUID();
+ assert Objects.equals(newClusterUUID, prevClusterUUID) : "Full restart not successful. cluster uuid has changed";
+
+ long newClusterStateVersion = clusterService().state().version();
+ assert prevClusterStateVersion < newClusterStateVersion : String.format(
+ Locale.ROOT,
+ "ClusterState version is not restored. previousClusterVersion: [%s] is greater than current [%s]",
+ prevClusterStateVersion,
+ newClusterStateVersion
+ );
+ validateCurrentMetadata();
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, true);
+ }
- future = PlainActionFuture.newFuture();
+ private void validateMetadata(List indexNames) {
+ assertEquals(clusterService().state().metadata().indices().size(), indexNames.size());
+ for (String indexName : indexNames) {
+ assertTrue(clusterService().state().metadata().hasIndex(indexName));
+ }
+ }
- // Test -4 Step - 3 Trigger full cluster restore and validate fails
- restoreAndValidateFails(
- prevClusterUUID,
- future,
- IllegalStateException.class,
- "cannot restore index [remote-store-test-idx-1] because an open index with same name/uuid already exists in the cluster"
+ private void validateCurrentMetadata() throws Exception {
+ RemoteClusterStateService remoteClusterStateService = internalCluster().getInstance(
+ RemoteClusterStateService.class,
+ internalCluster().getClusterManagerName()
);
+ assertBusy(() -> {
+ ClusterMetadataManifest manifest = remoteClusterStateService.getLatestClusterMetadataManifest(
+ getClusterState().getClusterName().value(),
+ getClusterState().metadata().clusterUUID()
+ ).get();
+ ClusterState clusterState = getClusterState();
+ Metadata currentMetadata = clusterState.metadata();
+ assertEquals(currentMetadata.indices().size(), manifest.getIndices().size());
+ assertEquals(currentMetadata.coordinationMetadata().term(), manifest.getClusterTerm());
+ assertEquals(clusterState.version(), manifest.getStateVersion());
+ assertEquals(clusterState.stateUUID(), manifest.getStateUUID());
+ assertEquals(currentMetadata.clusterUUIDCommitted(), manifest.isClusterUUIDCommitted());
+ for (UploadedIndexMetadata uploadedIndexMetadata : manifest.getIndices()) {
+ IndexMetadata currentIndexMetadata = currentMetadata.index(uploadedIndexMetadata.getIndexName());
+ assertEquals(currentIndexMetadata.getIndex().getUUID(), uploadedIndexMetadata.getIndexUUID());
+ }
+ });
+ }
- // Test -4 Step - 4 validation restore is successful.
- ensureGreen(INDEX_NAME);
- // End of Test - 4
+ public void testDataStreamPostRemoteStateRestore() throws Exception {
+ new DataStreamRolloverIT() {
+ protected boolean triggerRemoteStateRestore() {
+ return true;
+ }
+ }.testDataStreamRollover();
}
- @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9834")
- public void testFullClusterRestoreManifestFilePointsToInvalidIndexMetadataPathThrowsException() throws Exception {
+ public void testFullClusterRestoreGlobalMetadata() throws Exception {
int shardCount = randomIntBetween(1, 2);
int replicaCount = 1;
int dataNodeCount = shardCount * (replicaCount + 1);
int clusterManagerNodeCount = 1;
// Step - 1 index some data to generate files in remote directory
- initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount);
-
+ Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, 1);
String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+ long prevClusterStateVersion = clusterService().state().version();
+
+ // Create global metadata - register a custom repo
+ Path repoPath = registerCustomRepository();
+
+ // Create global metadata - persistent settings
+ updatePersistentSettings(Settings.builder().put(SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey(), 34).build());
+
+ // Create global metadata - index template
+ putIndexTemplate();
+
+ // Create global metadata - Put cluster block
+ addClusterLevelReadOnlyBlock();
// Step - 2 Replace all nodes in the cluster with new nodes. This ensures new cluster state doesn't have previous index metadata
resetCluster(dataNodeCount, clusterManagerNodeCount);
@@ -233,56 +365,102 @@ public void testFullClusterRestoreManifestFilePointsToInvalidIndexMetadataPathTh
String newClusterUUID = clusterService().state().metadata().clusterUUID();
assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
- // Step - 4 Delete index metadata file in remote
- try {
- Files.move(
- segmentRepoPath.resolve(
- RemoteClusterStateService.encodeString(clusterService().state().getClusterName().value())
- + "/cluster-state/"
- + prevClusterUUID
- + "/index"
- ),
- segmentRepoPath.resolve("cluster-state/")
- );
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ // Step - 3 validate cluster state restored
+ long newClusterStateVersion = clusterService().state().version();
+ assert prevClusterStateVersion < newClusterStateVersion : String.format(
+ Locale.ROOT,
+ "ClusterState version is not restored. previousClusterVersion: [%s] is greater than current [%s]",
+ prevClusterStateVersion,
+ newClusterStateVersion
+ );
+
+ validateCurrentMetadata();
+ assertEquals(Integer.valueOf(34), SETTING_CLUSTER_MAX_SHARDS_PER_NODE.get(clusterService().state().metadata().settings()));
+ assertEquals(true, SETTING_READ_ONLY_SETTING.get(clusterService().state().metadata().settings()));
+ assertTrue(clusterService().state().blocks().hasGlobalBlock(CLUSTER_READ_ONLY_BLOCK));
+ // Remote the cluster read only block to ensure proper cleanup
+ updatePersistentSettings(Settings.builder().put(SETTING_READ_ONLY_SETTING.getKey(), false).build());
+ assertFalse(clusterService().state().blocks().hasGlobalBlock(CLUSTER_READ_ONLY_BLOCK));
- // Step - 5 Trigger full cluster restore and validate fails
- PlainActionFuture future = PlainActionFuture.newFuture();
- restoreAndValidateFails(prevClusterUUID, future, IllegalStateException.class, "asdsa");
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, false);
+
+ // validate global metadata restored
+ verifyRestoredRepositories(repoPath);
+ verifyRestoredIndexTemplate();
}
- private void reduceShardLimits(int maxShardsPerNode, int maxShardsPerCluster) {
- // Step 3 - Reduce shard limits to hit shard limit with less no of shards
- try {
+ private Path registerCustomRepository() {
+ Path path = randomRepoPath();
+ assertAcked(
client().admin()
.cluster()
- .updateSettings(
- new ClusterUpdateSettingsRequest().transientSettings(
- Settings.builder()
- .put(SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey(), maxShardsPerNode)
- .put(SETTING_MAX_SHARDS_PER_CLUSTER_KEY, maxShardsPerCluster)
- )
- )
- .get();
- } catch (InterruptedException | ExecutionException e) {
- throw new RuntimeException(e);
- }
+ .preparePutRepository("custom-repo")
+ .setType("fs")
+ .setSettings(Settings.builder().put("location", path).put("compress", false))
+ .get()
+ );
+ return path;
}
- private void resetShardLimits() {
- // Step - 5 Reset the cluster settings
+ private void verifyRestoredRepositories(Path repoPath) {
+ RepositoriesMetadata repositoriesMetadata = clusterService().state().metadata().custom(RepositoriesMetadata.TYPE);
+ assertEquals(3, repositoriesMetadata.repositories().size()); // includes remote store repo as well
+ assertTrue(SYSTEM_REPOSITORY_SETTING.get(repositoriesMetadata.repository(REPOSITORY_NAME).settings()));
+ assertTrue(SYSTEM_REPOSITORY_SETTING.get(repositoriesMetadata.repository(REPOSITORY_2_NAME).settings()));
+ assertEquals("fs", repositoriesMetadata.repository("custom-repo").type());
+ assertEquals(
+ Settings.builder().put("location", repoPath).put("compress", false).build(),
+ repositoriesMetadata.repository("custom-repo").settings()
+ );
+
+ // repo cleanup post verification
+ clusterAdmin().prepareDeleteRepository("custom-repo").get();
+ }
+
+ private void addClusterLevelReadOnlyBlock() throws InterruptedException, ExecutionException {
+ updatePersistentSettings(Settings.builder().put(SETTING_READ_ONLY_SETTING.getKey(), true).build());
+ assertTrue(clusterService().state().blocks().hasGlobalBlock(CLUSTER_READ_ONLY_BLOCK));
+ }
+
+ private void updatePersistentSettings(Settings settings) throws ExecutionException, InterruptedException {
ClusterUpdateSettingsRequest resetRequest = new ClusterUpdateSettingsRequest();
- resetRequest.transientSettings(
- Settings.builder().putNull(SETTING_CLUSTER_MAX_SHARDS_PER_NODE.getKey()).putNull(SETTING_MAX_SHARDS_PER_CLUSTER_KEY)
+ resetRequest.persistentSettings(settings);
+ assertAcked(client().admin().cluster().updateSettings(resetRequest).get());
+ }
+
+ private void verifyRestoredIndexTemplate() {
+ Map indexTemplateMetadataMap = clusterService().state().metadata().templates();
+ assertEquals(1, indexTemplateMetadataMap.size());
+ assertEquals(Arrays.asList("pattern-1", "log-*"), indexTemplateMetadataMap.get("my-template").patterns());
+ assertEquals(
+ Settings.builder() // <1>
+ .put("index.number_of_shards", 3)
+ .put("index.number_of_replicas", 1)
+ .build(),
+ indexTemplateMetadataMap.get("my-template").settings()
);
+ }
- try {
- client().admin().cluster().updateSettings(resetRequest).get();
- } catch (InterruptedException | ExecutionException e) {
- throw new RuntimeException(e);
- }
+ private static void putIndexTemplate() {
+ PutIndexTemplateRequest request = new PutIndexTemplateRequest("my-template"); // <1>
+ request.patterns(Arrays.asList("pattern-1", "log-*")); // <2>
+
+ request.settings(
+ Settings.builder() // <1>
+ .put("index.number_of_shards", 3)
+ .put("index.number_of_replicas", 1)
+ );
+ assertTrue(client().admin().indices().putTemplate(request).actionGet().isAcknowledged());
}
+ private static void updateIndexBlock(boolean value, String secondIndexName) throws InterruptedException, ExecutionException {
+ assertAcked(
+ client().admin()
+ .indices()
+ .updateSettings(
+ new UpdateSettingsRequest(Settings.builder().put(INDEX_READ_ONLY_SETTING.getKey(), value).build(), secondIndexName)
+ )
+ .get()
+ );
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java
index 1fb5c2052aded..b3b4f8e10fd31 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java
@@ -509,4 +509,27 @@ public void testRestoreSnapshotToIndexWithSameNameDifferentUUID() throws Excepti
assertHitCount(client(dataNodes.get(1)).prepareSearch(INDEX_NAME).setSize(0).get(), 50);
});
}
+
+ public void testNoSearchIdleForAnyReplicaCount() throws ExecutionException, InterruptedException {
+ internalCluster().startClusterManagerOnlyNode();
+ String primaryShardNode = internalCluster().startDataOnlyNodes(1).get(0);
+
+ createIndex(INDEX_NAME, remoteStoreIndexSettings(0));
+ ensureGreen(INDEX_NAME);
+ IndexShard indexShard = getIndexShard(primaryShardNode);
+ assertFalse(indexShard.isSearchIdleSupported());
+
+ String replicaShardNode = internalCluster().startDataOnlyNodes(1).get(0);
+ assertAcked(
+ client().admin()
+ .indices()
+ .prepareUpdateSettings(INDEX_NAME)
+ .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1))
+ );
+ ensureGreen(INDEX_NAME);
+ assertFalse(indexShard.isSearchIdleSupported());
+
+ indexShard = getIndexShard(replicaShardNode);
+ assertFalse(indexShard.isSearchIdleSupported());
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java
index b97e93f323fb2..acdb21d072320 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java
@@ -28,9 +28,13 @@
public class RemoteStoreRefreshListenerIT extends AbstractRemoteStoreMockRepositoryIntegTestCase {
public void testRemoteRefreshRetryOnFailure() throws Exception {
-
Path location = randomRepoPath().toAbsolutePath();
setup(location, randomDoubleBetween(0.1, 0.15, true), "metadata", 10L);
+ client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setPersistentSettings(Settings.builder().put(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), false))
+ .get();
// Here we are having flush/refresh after each iteration of indexing. However, the refresh will not always succeed
// due to IOExceptions that are thrown while doing uploadBlobs.
@@ -56,7 +60,7 @@ public void testRemoteRefreshRetryOnFailure() throws Exception {
logger.info("Local files = {}, Repo files = {}", sortedFilesInLocal, sortedFilesInRepo);
assertTrue(filesInRepo.containsAll(filesInLocal));
}, 90, TimeUnit.SECONDS);
- deleteRepo();
+ cleanupRepo();
}
public void testRemoteRefreshSegmentPressureSettingChanged() {
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java
index 4d56a1e94e3fc..ef2dcf3217df6 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java
@@ -8,17 +8,31 @@
package org.opensearch.remotestore;
+import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesAction;
+import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest;
+import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse;
+import org.opensearch.client.Client;
+import org.opensearch.cluster.metadata.RepositoryMetadata;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.xcontent.XContentType;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.core.xcontent.ToXContent;
+import org.opensearch.core.xcontent.XContentBuilder;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.disruption.NetworkDisruption;
import org.opensearch.test.transport.MockTransportService;
+import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;
+import static org.opensearch.repositories.blobstore.BlobStoreRepository.SYSTEM_REPOSITORY_SETTING;
+
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RemoteStoreRepositoryRegistrationIT extends RemoteStoreBaseIntegTestCase {
@@ -94,4 +108,77 @@ public void testMultiNodeClusterRandomNodeRecoverNetworkIsolation() {
internalCluster().clearDisruptionScheme();
}
+
+ public void testMultiNodeClusterRandomNodeRecoverNetworkIsolationPostNonRestrictedSettingsUpdate() {
+ Set nodesInOneSide = internalCluster().startNodes(3).stream().collect(Collectors.toCollection(HashSet::new));
+ Set nodesInAnotherSide = internalCluster().startNodes(3).stream().collect(Collectors.toCollection(HashSet::new));
+ ensureStableCluster(6);
+
+ NetworkDisruption networkDisruption = new NetworkDisruption(
+ new NetworkDisruption.TwoPartitions(nodesInOneSide, nodesInAnotherSide),
+ NetworkDisruption.DISCONNECT
+ );
+ internalCluster().setDisruptionScheme(networkDisruption);
+
+ networkDisruption.startDisrupting();
+
+ final Client client = client(nodesInOneSide.iterator().next());
+ RepositoryMetadata repositoryMetadata = client.admin()
+ .cluster()
+ .prepareGetRepositories(REPOSITORY_NAME)
+ .get()
+ .repositories()
+ .get(0);
+ Settings.Builder updatedSettings = Settings.builder().put(repositoryMetadata.settings()).put("chunk_size", new ByteSizeValue(20));
+ updatedSettings.remove("system_repository");
+
+ client.admin()
+ .cluster()
+ .preparePutRepository(repositoryMetadata.name())
+ .setType(repositoryMetadata.type())
+ .setSettings(updatedSettings)
+ .get();
+
+ ensureStableCluster(3, nodesInOneSide.stream().findAny().get());
+ networkDisruption.stopDisrupting();
+
+ ensureStableCluster(6);
+
+ internalCluster().clearDisruptionScheme();
+ }
+
+ public void testNodeRestartPostNonRestrictedSettingsUpdate() throws Exception {
+ internalCluster().startClusterManagerOnlyNode();
+ internalCluster().startNodes(3);
+
+ final Client client = client();
+ RepositoryMetadata repositoryMetadata = client.admin()
+ .cluster()
+ .prepareGetRepositories(REPOSITORY_NAME)
+ .get()
+ .repositories()
+ .get(0);
+ Settings.Builder updatedSettings = Settings.builder().put(repositoryMetadata.settings()).put("chunk_size", new ByteSizeValue(20));
+ updatedSettings.remove("system_repository");
+
+ client.admin()
+ .cluster()
+ .preparePutRepository(repositoryMetadata.name())
+ .setType(repositoryMetadata.type())
+ .setSettings(updatedSettings)
+ .get();
+
+ internalCluster().restartRandomDataNode();
+
+ ensureStableCluster(4);
+ }
+
+ public void testSystemRepositorySettingIsHiddenForGetRepositoriesRequest() throws IOException {
+ GetRepositoriesRequest request = new GetRepositoriesRequest(new String[] { REPOSITORY_NAME });
+ GetRepositoriesResponse repositoriesResponse = client().execute(GetRepositoriesAction.INSTANCE, request).actionGet();
+ XContentBuilder builder = MediaTypeRegistry.contentBuilder(randomFrom(XContentType.JSON));
+ XContentBuilder xContentBuilder = repositoriesResponse.toXContent(builder, ToXContent.EMPTY_PARAMS);
+ repositoriesResponse = GetRepositoriesResponse.fromXContent(createParser(xContentBuilder));
+ assertEquals(false, SYSTEM_REPOSITORY_SETTING.get(repositoriesResponse.repositories().get(0).settings()));
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java
index 65335f444a2df..7626e3dba6424 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java
@@ -12,17 +12,26 @@
import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreResponse;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.cluster.health.ClusterHealthStatus;
+import org.opensearch.cluster.node.DiscoveryNode;
+import org.opensearch.cluster.service.ClusterService;
+import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.common.unit.ByteSizeUnit;
import org.opensearch.repositories.RepositoriesService;
+import org.opensearch.repositories.Repository;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
import java.io.IOException;
+import java.nio.file.Path;
import java.util.HashMap;
+import java.util.Locale;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.greaterThan;
@@ -388,14 +397,41 @@ public void testRTSRestoreDataOnlyInTranslog() throws Exception {
public void testRateLimitedRemoteDownloads() throws Exception {
clusterSettingsSuppliedByTest = true;
int shardCount = randomIntBetween(1, 3);
+ Path segmentRepoPath = randomRepoPath();
+ Path tlogRepoPath = randomRepoPath();
prepareCluster(
1,
3,
INDEX_NAME,
0,
shardCount,
- buildRemoteStoreNodeAttributes(REPOSITORY_NAME, randomRepoPath(), REPOSITORY_2_NAME, randomRepoPath(), true)
+ buildRemoteStoreNodeAttributes(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, tlogRepoPath, true)
);
+
+ // validate inplace repository metadata update
+ ClusterService clusterService = internalCluster().getInstance(ClusterService.class);
+ DiscoveryNode node = clusterService.localNode();
+ String settingsAttributeKeyPrefix = String.format(
+ Locale.getDefault(),
+ REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX,
+ REPOSITORY_NAME
+ );
+ Map settingsMap = node.getAttributes()
+ .keySet()
+ .stream()
+ .filter(key -> key.startsWith(settingsAttributeKeyPrefix))
+ .collect(Collectors.toMap(key -> key.replace(settingsAttributeKeyPrefix, ""), key -> node.getAttributes().get(key)));
+ Settings.Builder settings = Settings.builder();
+ settingsMap.entrySet().forEach(entry -> settings.put(entry.getKey(), entry.getValue()));
+ settings.put("location", segmentRepoPath).put("max_remote_download_bytes_per_sec", 4, ByteSizeUnit.KB);
+
+ assertAcked(client().admin().cluster().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(settings).get());
+
+ for (RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) {
+ Repository segmentRepo = repositoriesService.repository(REPOSITORY_NAME);
+ assertEquals("4096b", segmentRepo.getMetadata().settings().get("max_remote_download_bytes_per_sec"));
+ }
+
Map indexStats = indexData(5, false, INDEX_NAME);
assertEquals(shardCount, getNumShards(INDEX_NAME).totalNumShards);
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(INDEX_NAME)));
@@ -414,6 +450,15 @@ public void testRateLimitedRemoteDownloads() throws Exception {
assertEquals(shardCount, getNumShards(INDEX_NAME).totalNumShards);
assertEquals(0, getNumShards(INDEX_NAME).numReplicas);
verifyRestoredData(indexStats, INDEX_NAME);
+
+ // revert repo metadata to pass asserts on repo metadata vs. node attrs during teardown
+ // https://github.com/opensearch-project/OpenSearch/pull/9569#discussion_r1345668700
+ settings.remove("max_remote_download_bytes_per_sec");
+ assertAcked(client().admin().cluster().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(settings).get());
+ for (RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) {
+ Repository segmentRepo = repositoriesService.repository(REPOSITORY_NAME);
+ assertNull(segmentRepo.getMetadata().settings().get("max_remote_download_bytes_per_sec"));
+ }
}
// TODO: Restore flow - index aliases
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java
index 8ae25c6758195..2d3ab135d0377 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java
@@ -15,6 +15,8 @@
import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsResponse;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.cluster.ClusterState;
+import org.opensearch.cluster.coordination.FollowersChecker;
+import org.opensearch.cluster.coordination.LeaderChecker;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.cluster.routing.ShardRoutingState;
@@ -23,15 +25,20 @@
import org.opensearch.index.IndexSettings;
import org.opensearch.index.remote.RemoteSegmentTransferTracker;
import org.opensearch.index.remote.RemoteTranslogTransferTracker;
+import org.opensearch.plugins.Plugin;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
-import org.junit.Before;
+import org.opensearch.test.disruption.NetworkDisruption;
+import org.opensearch.test.transport.MockTransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
import java.util.Locale;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -44,12 +51,17 @@ public class RemoteStoreStatsIT extends RemoteStoreBaseIntegTestCase {
private static final String INDEX_NAME = "remote-store-test-idx-1";
- @Before
+ @Override
+ protected Collection> nodePlugins() {
+ return Arrays.asList(MockTransportService.TestPlugin.class);
+ }
+
public void setup() {
internalCluster().startNodes(3);
}
public void testStatsResponseFromAllNodes() {
+ setup();
// Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes
// during this time frame. This ensures that the segment upload has started.
@@ -118,6 +130,7 @@ public void testStatsResponseFromAllNodes() {
}
public void testStatsResponseAllShards() {
+ setup();
// Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes
// during this time frame. This ensures that the segment upload has started.
@@ -175,6 +188,7 @@ public void testStatsResponseAllShards() {
}
public void testStatsResponseFromLocalNode() {
+ setup();
// Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes
// during this time frame. This ensures that the segment upload has started.
@@ -236,6 +250,7 @@ public void testStatsResponseFromLocalNode() {
}
public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exception {
+ setup();
// Scenario:
// - Create index with single primary and single replica shard
// - Disable Refresh Interval for the index
@@ -325,6 +340,7 @@ public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exce
}
public void testDownloadStatsCorrectnessSinglePrimaryMultipleReplicaShards() throws Exception {
+ setup();
// Scenario:
// - Create index with single primary and N-1 replica shards (N = no of data nodes)
// - Disable Refresh Interval for the index
@@ -416,6 +432,7 @@ public void testDownloadStatsCorrectnessSinglePrimaryMultipleReplicaShards() thr
}
public void testStatsOnShardRelocation() {
+ setup();
// Scenario:
// - Create index with single primary and single replica shard
// - Index documents
@@ -471,6 +488,7 @@ public void testStatsOnShardRelocation() {
}
public void testStatsOnShardUnassigned() throws IOException {
+ setup();
// Scenario:
// - Create index with single primary and two replica shard
// - Index documents
@@ -497,6 +515,7 @@ public void testStatsOnShardUnassigned() throws IOException {
}
public void testStatsOnRemoteStoreRestore() throws IOException {
+ setup();
// Creating an index with primary shard count == total nodes in cluster and 0 replicas
int dataNodeCount = client().admin().cluster().prepareHealth().get().getNumberOfDataNodes();
createIndex(INDEX_NAME, remoteStoreIndexSettings(0, dataNodeCount));
@@ -544,6 +563,7 @@ public void testStatsOnRemoteStoreRestore() throws IOException {
}
public void testNonZeroPrimaryStatsOnNewlyCreatedIndexWithZeroDocs() throws Exception {
+ setup();
// Create an index with one primary and one replica shard
createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 1));
ensureGreen(INDEX_NAME);
@@ -561,26 +581,103 @@ public void testNonZeroPrimaryStatsOnNewlyCreatedIndexWithZeroDocs() throws Exce
.getRemoteStoreStats();
Arrays.stream(remoteStoreStats).forEach(statObject -> {
RemoteSegmentTransferTracker.Stats segmentStats = statObject.getSegmentStats();
+ RemoteTranslogTransferTracker.Stats translogStats = statObject.getTranslogStats();
if (statObject.getShardRouting().primary()) {
assertTrue(
segmentStats.totalUploadsSucceeded == 1
&& segmentStats.totalUploadsStarted == segmentStats.totalUploadsSucceeded
&& segmentStats.totalUploadsFailed == 0
);
+ // On primary shard creation, we upload to remote translog post primary mode activation.
+ // This changes upload stats to non-zero for primary shard.
+ assertNonZeroTranslogUploadStatsNoFailures(translogStats);
} else {
assertTrue(
segmentStats.directoryFileTransferTrackerStats.transferredBytesStarted == 0
&& segmentStats.directoryFileTransferTrackerStats.transferredBytesSucceeded == 0
);
+ assertZeroTranslogUploadStats(translogStats);
}
-
- RemoteTranslogTransferTracker.Stats translogStats = statObject.getTranslogStats();
- assertZeroTranslogUploadStats(translogStats);
assertZeroTranslogDownloadStats(translogStats);
});
}, 5, TimeUnit.SECONDS);
}
+ public void testStatsCorrectnessOnFailover() {
+ Settings clusterSettings = Settings.builder()
+ .put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "100ms")
+ .put(LeaderChecker.LEADER_CHECK_INTERVAL_SETTING.getKey(), "500ms")
+ .put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 1)
+ .put(FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING.getKey(), "100ms")
+ .put(FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING.getKey(), "500ms")
+ .put(FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING.getKey(), 1)
+ .put(nodeSettings(0))
+ .build();
+ String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(clusterSettings);
+ internalCluster().startDataOnlyNodes(2, clusterSettings);
+
+ // Create an index with one primary and one replica shard
+ createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 1));
+ ensureGreen(INDEX_NAME);
+
+ // Index some docs and refresh
+ indexDocs();
+ refresh(INDEX_NAME);
+
+ String primaryNode = primaryNodeName(INDEX_NAME);
+ String replicaNode = replicaNodeName(INDEX_NAME);
+
+ // Start network disruption - primary node will be isolated
+ Set nodesInOneSide = Stream.of(clusterManagerNode, replicaNode).collect(Collectors.toCollection(HashSet::new));
+ Set nodesInOtherSide = Stream.of(primaryNode).collect(Collectors.toCollection(HashSet::new));
+ NetworkDisruption networkDisruption = new NetworkDisruption(
+ new NetworkDisruption.TwoPartitions(nodesInOneSide, nodesInOtherSide),
+ NetworkDisruption.DISCONNECT
+ );
+ internalCluster().setDisruptionScheme(networkDisruption);
+ logger.info("--> network disruption is started");
+ networkDisruption.startDisrupting();
+ ensureStableCluster(2, clusterManagerNode);
+
+ RemoteStoreStatsResponse response = client(clusterManagerNode).admin().cluster().prepareRemoteStoreStats(INDEX_NAME, "0").get();
+ final String indexShardId = String.format(Locale.ROOT, "[%s][%s]", INDEX_NAME, "0");
+ List matches = Arrays.stream(response.getRemoteStoreStats())
+ .filter(stat -> indexShardId.equals(stat.getSegmentStats().shardId.toString()))
+ .collect(Collectors.toList());
+ assertEquals(1, matches.size());
+ RemoteSegmentTransferTracker.Stats segmentStats = matches.get(0).getSegmentStats();
+ assertEquals(0, segmentStats.refreshTimeLagMs);
+
+ networkDisruption.stopDisrupting();
+ internalCluster().clearDisruptionScheme();
+ ensureStableCluster(3, clusterManagerNode);
+ ensureGreen(INDEX_NAME);
+ logger.info("Test completed");
+ }
+
+ public void testZeroLagOnCreateIndex() throws InterruptedException {
+ setup();
+ String clusterManagerNode = internalCluster().getClusterManagerName();
+
+ int numOfShards = randomIntBetween(1, 3);
+ createIndex(INDEX_NAME, remoteStoreIndexSettings(1, numOfShards));
+ ensureGreen(INDEX_NAME);
+ long currentTimeNs = System.nanoTime();
+ while (currentTimeNs == System.nanoTime()) {
+ Thread.sleep(10);
+ }
+
+ for (int i = 0; i < numOfShards; i++) {
+ RemoteStoreStatsResponse response = client(clusterManagerNode).admin()
+ .cluster()
+ .prepareRemoteStoreStats(INDEX_NAME, String.valueOf(i))
+ .get();
+ for (RemoteStoreStats remoteStoreStats : response.getRemoteStoreStats()) {
+ assertEquals(0, remoteStoreStats.getSegmentStats().refreshTimeLagMs);
+ }
+ }
+ }
+
private void indexDocs() {
for (int i = 0; i < randomIntBetween(5, 10); i++) {
if (randomBoolean()) {
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreDisruptionIT.java
new file mode 100644
index 0000000000000..b7b3f1d14f422
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreDisruptionIT.java
@@ -0,0 +1,133 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.remotestore;
+
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.index.Index;
+import org.opensearch.index.IndexService;
+import org.opensearch.index.shard.IndexShard;
+import org.opensearch.indices.IndicesService;
+import org.opensearch.indices.replication.SegmentReplicationState;
+import org.opensearch.indices.replication.SegmentReplicationTarget;
+import org.opensearch.indices.replication.SegmentReplicationTargetService;
+import org.opensearch.indices.replication.common.ReplicationCollection;
+import org.opensearch.test.InternalTestCluster;
+import org.opensearch.test.OpenSearchIntegTestCase;
+
+import java.nio.file.Path;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * This class runs tests with remote store + segRep while blocking file downloads
+ */
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
+public class SegmentReplicationUsingRemoteStoreDisruptionIT extends AbstractRemoteStoreMockRepositoryIntegTestCase {
+
+ @Override
+ public Settings indexSettings() {
+ return remoteStoreIndexSettings(1);
+ }
+
+ @Override
+ protected boolean addMockInternalEngine() {
+ return false;
+ }
+
+ public void testCancelReplicationWhileSyncingSegments() throws Exception {
+ Path location = randomRepoPath().toAbsolutePath();
+ setup(location, 0d, "metadata", Long.MAX_VALUE, 1);
+
+ final Set dataNodeNames = internalCluster().getDataNodeNames();
+ final String replicaNode = getNode(dataNodeNames, false);
+ final String primaryNode = getNode(dataNodeNames, true);
+
+ SegmentReplicationTargetService targetService = internalCluster().getInstance(SegmentReplicationTargetService.class, replicaNode);
+ ensureGreen(INDEX_NAME);
+ blockNodeOnAnySegmentFile(REPOSITORY_NAME, replicaNode);
+ final IndexShard indexShard = getIndexShard(replicaNode, INDEX_NAME);
+ indexSingleDoc();
+ refresh(INDEX_NAME);
+ waitForBlock(replicaNode, REPOSITORY_NAME, TimeValue.timeValueSeconds(10));
+ final SegmentReplicationState state = targetService.getOngoingEventSegmentReplicationState(indexShard.shardId());
+ assertEquals(SegmentReplicationState.Stage.GET_FILES, state.getStage());
+ ReplicationCollection.ReplicationRef segmentReplicationTargetReplicationRef = targetService.get(
+ state.getReplicationId()
+ );
+ final SegmentReplicationTarget segmentReplicationTarget = segmentReplicationTargetReplicationRef.get();
+ // close the target ref here otherwise it will hold a refcount
+ segmentReplicationTargetReplicationRef.close();
+ assertNotNull(segmentReplicationTarget);
+ assertTrue(segmentReplicationTarget.refCount() > 0);
+ internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
+ assertBusy(() -> {
+ assertTrue(indexShard.routingEntry().primary());
+ assertNull(targetService.getOngoingEventSegmentReplicationState(indexShard.shardId()));
+ assertEquals("Target should be closed", 0, segmentReplicationTarget.refCount());
+ });
+ unblockNode(REPOSITORY_NAME, replicaNode);
+ cleanupRepo();
+ }
+
+ public void testCancelReplicationWhileFetchingMetadata() throws Exception {
+ Path location = randomRepoPath().toAbsolutePath();
+ setup(location, 0d, "metadata", Long.MAX_VALUE, 1);
+
+ final Set dataNodeNames = internalCluster().getDataNodeNames();
+ final String replicaNode = getNode(dataNodeNames, false);
+ final String primaryNode = getNode(dataNodeNames, true);
+
+ SegmentReplicationTargetService targetService = internalCluster().getInstance(SegmentReplicationTargetService.class, replicaNode);
+ ensureGreen(INDEX_NAME);
+ blockNodeOnAnyFiles(REPOSITORY_NAME, replicaNode);
+ final IndexShard indexShard = getIndexShard(replicaNode, INDEX_NAME);
+ indexSingleDoc();
+ refresh(INDEX_NAME);
+ waitForBlock(replicaNode, REPOSITORY_NAME, TimeValue.timeValueSeconds(10));
+ final SegmentReplicationState state = targetService.getOngoingEventSegmentReplicationState(indexShard.shardId());
+ assertEquals(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO, state.getStage());
+ ReplicationCollection.ReplicationRef segmentReplicationTargetReplicationRef = targetService.get(
+ state.getReplicationId()
+ );
+ final SegmentReplicationTarget segmentReplicationTarget = segmentReplicationTargetReplicationRef.get();
+ // close the target ref here otherwise it will hold a refcount
+ segmentReplicationTargetReplicationRef.close();
+ assertNotNull(segmentReplicationTarget);
+ assertTrue(segmentReplicationTarget.refCount() > 0);
+ internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode));
+ assertBusy(() -> {
+ assertTrue(indexShard.routingEntry().primary());
+ assertNull(targetService.getOngoingEventSegmentReplicationState(indexShard.shardId()));
+ assertEquals("Target should be closed", 0, segmentReplicationTarget.refCount());
+ });
+ unblockNode(REPOSITORY_NAME, replicaNode);
+ cleanupRepo();
+ }
+
+ private String getNode(Set dataNodeNames, boolean primary) {
+ assertEquals(2, dataNodeNames.size());
+ for (String name : dataNodeNames) {
+ final IndexShard indexShard = getIndexShard(name, INDEX_NAME);
+ if (indexShard.routingEntry().primary() == primary) {
+ return name;
+ }
+ }
+ return null;
+ }
+
+ private IndexShard getIndexShard(String node, String indexName) {
+ final Index index = resolveIndex(indexName);
+ IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node);
+ IndexService indexService = indicesService.indexService(index);
+ assertNotNull(indexService);
+ final Optional shardId = indexService.shardIds().stream().findFirst();
+ return shardId.map(indexService::getShard).orElse(null);
+ }
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java
index 45c3ef7f5bae5..23864c35ad154 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java
@@ -17,7 +17,6 @@
import java.nio.file.Path;
import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
/**
* This class runs Segment Replication Integ test suite with remote store enabled.
@@ -50,6 +49,6 @@ public void setup() {
@After
public void teardown() {
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).get();
}
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java
index 0da4d81a8871e..6cfc76b7e3223 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java
@@ -17,7 +17,6 @@
import java.nio.file.Path;
import static org.opensearch.remotestore.RemoteStoreBaseIntegTestCase.remoteStoreClusterSettings;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
/**
* This class executes the SegmentReplicationPressureIT suite with remote store integration enabled.
@@ -49,6 +48,6 @@ public void setup() {
@After
public void teardown() {
- assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
+ clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).get();
}
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java
index 98fab139f4902..3dfde6f472525 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java
@@ -108,8 +108,15 @@ public RepositoryMetadata buildRepositoryMetadata(DiscoveryNode node, String nam
}
public void testRateLimitedRemoteUploads() throws Exception {
+ clusterSettingsSuppliedByTest = true;
overrideBuildRepositoryMetadata = true;
- internalCluster().startNode();
+ Settings.Builder clusterSettings = Settings.builder()
+ .put(remoteStoreClusterSettings(REPOSITORY_NAME, repositoryLocation, REPOSITORY_2_NAME, repositoryLocation));
+ clusterSettings.put(
+ String.format(Locale.getDefault(), "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, REPOSITORY_NAME),
+ MockFsRepositoryPlugin.TYPE
+ );
+ internalCluster().startNode(clusterSettings.build());
Client client = client();
logger.info("--> updating repository");
assertAcked(
@@ -119,7 +126,6 @@ public void testRateLimitedRemoteUploads() throws Exception {
.setType(MockFsRepositoryPlugin.TYPE)
.setSettings(
Settings.builder()
- .put(BlobStoreRepository.SYSTEM_REPOSITORY_SETTING.getKey(), true)
.put("location", repositoryLocation)
.put("compress", compress)
.put("max_remote_upload_bytes_per_sec", "1kb")
diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsAsyncBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsAsyncBlobContainer.java
index 079753de95680..36987ac2d4991 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsAsyncBlobContainer.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsAsyncBlobContainer.java
@@ -27,6 +27,7 @@
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
@@ -124,11 +125,11 @@ public void readBlobAsync(String blobName, ActionListener listener)
long contentLength = listBlobs().get(blobName).length();
long partSize = contentLength / 10;
int numberOfParts = (int) ((contentLength % partSize) == 0 ? contentLength / partSize : (contentLength / partSize) + 1);
- List blobPartStreams = new ArrayList<>();
+ List blobPartStreams = new ArrayList<>();
for (int partNumber = 0; partNumber < numberOfParts; partNumber++) {
long offset = partNumber * partSize;
InputStreamContainer blobPartStream = new InputStreamContainer(readBlob(blobName, offset, partSize), partSize, offset);
- blobPartStreams.add(blobPartStream);
+ blobPartStreams.add(() -> CompletableFuture.completedFuture(blobPartStream));
}
ReadContext blobReadContext = new ReadContext(contentLength, blobPartStreams, null);
listener.onResponse(blobReadContext);
diff --git a/server/src/internalClusterTest/java/org/opensearch/repositories/RepositoriesServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/repositories/RepositoriesServiceIT.java
index f149d538cc47a..b8415f4b41815 100644
--- a/server/src/internalClusterTest/java/org/opensearch/repositories/RepositoriesServiceIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/repositories/RepositoriesServiceIT.java
@@ -108,4 +108,16 @@ public void testUpdateRepository() {
final Repository updatedRepository = repositoriesService.repository(repositoryName);
assertThat(updatedRepository, updated ? not(sameInstance(originalRepository)) : sameInstance(originalRepository));
}
+
+ public void testSystemRepositoryCantBeCreated() {
+ internalCluster();
+ final String repositoryName = "test-repo";
+ final Client client = client();
+ final Settings.Builder repoSettings = Settings.builder().put("system_repository", true).put("location", randomRepoPath());
+
+ assertThrows(
+ RepositoryException.class,
+ () -> client.admin().cluster().preparePutRepository(repositoryName).setType(FsRepository.TYPE).setSettings(repoSettings).get()
+ );
+ }
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/SearchTimeoutIT.java b/server/src/internalClusterTest/java/org/opensearch/search/SearchTimeoutIT.java
index 94816346e6c9e..52cc797ddd8da 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/SearchTimeoutIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/SearchTimeoutIT.java
@@ -93,6 +93,7 @@ public void testSimpleTimeout() throws Exception {
client().prepareIndex("test").setId(Integer.toString(i)).setSource("field", "value").get();
}
refresh("test");
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setTimeout(new TimeValue(5, TimeUnit.MILLISECONDS))
@@ -104,12 +105,11 @@ public void testSimpleTimeout() throws Exception {
}
public void testSimpleDoesNotTimeout() throws Exception {
- final int numDocs = 10;
+ final int numDocs = 9;
for (int i = 0; i < numDocs; i++) {
- client().prepareIndex("test").setId(Integer.toString(i)).setSource("field", "value").get();
+ client().prepareIndex("test").setId(Integer.toString(i)).setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
}
- refresh("test");
-
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setTimeout(new TimeValue(10000, TimeUnit.SECONDS))
.setQuery(scriptQuery(new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())))
@@ -122,7 +122,7 @@ public void testSimpleDoesNotTimeout() throws Exception {
public void testPartialResultsIntolerantTimeout() throws Exception {
client().prepareIndex("test").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
-
+ indexRandomForConcurrentSearch("test");
OpenSearchException ex = expectThrows(
OpenSearchException.class,
() -> client().prepareSearch("test")
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/SearchWeightedRoutingIT.java b/server/src/internalClusterTest/java/org/opensearch/search/SearchWeightedRoutingIT.java
index 5207dab83f1d9..aa1fe695ecc12 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/SearchWeightedRoutingIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/SearchWeightedRoutingIT.java
@@ -501,8 +501,9 @@ public void testShardRoutingWithNetworkDisruption_FailOpenEnabled() throws Excep
logger.info("--> creating network partition disruption");
final String clusterManagerNode1 = internalCluster().getClusterManagerName();
- Set nodesInOneSide = Stream.of(clusterManagerNode1, nodeMap.get("b").get(0)).collect(Collectors.toCollection(HashSet::new));
- Set nodesInOtherSide = Stream.of(nodeMap.get("a").get(0)).collect(Collectors.toCollection(HashSet::new));
+ Set nodesInOneSide = Stream.of(nodeMap.get("a").get(0)).collect(Collectors.toCollection(HashSet::new));
+ Set nodesInOtherSide = Stream.of(clusterManagerNode1, nodeMap.get("b").get(0), nodeMap.get("c").get(0))
+ .collect(Collectors.toCollection(HashSet::new));
NetworkDisruption networkDisruption = new NetworkDisruption(
new NetworkDisruption.TwoPartitions(nodesInOneSide, nodesInOtherSide),
@@ -870,8 +871,7 @@ private void assertSearchInAZ(String az) {
SearchStats.Stats searchStats = stat.getIndices().getSearch().getTotal();
if (stat.getNode().isDataNode()) {
if (stat.getNode().getId().equals(dataNodeId)) {
- Assert.assertTrue(searchStats.getFetchCount() > 0L);
- Assert.assertTrue(searchStats.getQueryCount() > 0L);
+ Assert.assertTrue(searchStats.getFetchCount() > 0L || searchStats.getQueryCount() > 0L);
}
}
}
@@ -945,7 +945,6 @@ public void testSearchAggregationWithNetworkDisruption_FailOpenEnabled() throws
}
logger.info("--> network disruption is stopped");
- networkDisruption.stopDisrupting();
for (int i = 0; i < 50; i++) {
try {
@@ -962,6 +961,8 @@ public void testSearchAggregationWithNetworkDisruption_FailOpenEnabled() throws
fail("search should not fail");
}
}
+ networkDisruption.stopDisrupting();
+
assertSearchInAZ("b");
assertSearchInAZ("c");
assertNoSearchInAZ("a");
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java
index 4ce8af3e0f081..ee94e574228df 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java
@@ -124,7 +124,7 @@ protected Settings featureFlagSettings() {
}
private ZonedDateTime date(String date) {
- return DateFormatters.from(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parse(date));
+ return DateFormatters.from(DateFieldMapper.getDefaultDateTimeFormatter().parse(date));
}
private static String format(ZonedDateTime date, String pattern) {
@@ -1481,7 +1481,7 @@ public void testExceptionOnNegativeInterval() {
/**
* https://github.com/elastic/elasticsearch/issues/31760 shows an edge case where an unmapped "date" field in two indices
* that are queried simultaneously can lead to the "format" parameter in the aggregation not being preserved correctly.
- *
+ *
* The error happens when the bucket from the "unmapped" index is received first in the reduce phase, however the case can
* be recreated when aggregating about a single index with an unmapped date field and also getting "empty" buckets.
*/
@@ -1624,8 +1624,8 @@ public void testScriptCaching() throws Exception {
.setSettings(Settings.builder().put("requests.cache.enable", true).put("number_of_shards", 1).put("number_of_replicas", 1))
.get()
);
- String date = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.format(date(1, 1));
- String date2 = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.format(date(2, 1));
+ String date = DateFieldMapper.getDefaultDateTimeFormatter().format(date(1, 1));
+ String date2 = DateFieldMapper.getDefaultDateTimeFormatter().format(date(2, 1));
indexRandom(
true,
client().prepareIndex("cache_test_idx").setId("1").setSource("d", date),
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramOffsetIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramOffsetIT.java
index 04115f69172da..d44071e1ef9c5 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramOffsetIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramOffsetIT.java
@@ -92,7 +92,7 @@ protected Settings featureFlagSettings() {
}
private ZonedDateTime date(String date) {
- return DateFormatters.from(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parse(date));
+ return DateFormatters.from(DateFieldMapper.getDefaultDateTimeFormatter().parse(date));
}
@Before
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DiversifiedSamplerIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DiversifiedSamplerIT.java
index 5e95073209c71..1d5f7f93e7410 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DiversifiedSamplerIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DiversifiedSamplerIT.java
@@ -33,9 +33,9 @@
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import org.opensearch.action.admin.indices.refresh.RefreshRequest;
import org.opensearch.action.search.SearchResponse;
import org.opensearch.action.search.SearchType;
+import org.opensearch.action.support.WriteRequest;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.FeatureFlags;
import org.opensearch.index.query.TermQueryBuilder;
@@ -132,13 +132,14 @@ public void setupSuiteScopeCluster() throws Exception {
client().prepareIndex("test")
.setId("" + i)
.setSource("author", parts[5], "name", parts[2], "genre", parts[8], "price", Float.parseFloat(parts[3]))
+ .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
client().prepareIndex("idx_unmapped_author")
.setId("" + i)
.setSource("name", parts[2], "genre", parts[8], "price", Float.parseFloat(parts[3]))
+ .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
}
- client().admin().indices().refresh(new RefreshRequest("test")).get();
}
public void testIssue10719() throws Exception {
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NaNSortingIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NaNSortingIT.java
index 1ef2c0e8db8c7..6289cd5e36151 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NaNSortingIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NaNSortingIT.java
@@ -172,6 +172,7 @@ public void setupSuiteScopeCluster() throws Exception {
client().prepareIndex("idx").setSource(source.endObject()).get();
}
refresh();
+ indexRandomForMultipleSlices("idx");
ensureSearchable();
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NestedIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NestedIT.java
index b3009ffcf4f45..7af2ac218800d 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NestedIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/NestedIT.java
@@ -224,6 +224,7 @@ public void setupSuiteScopeCluster() throws Exception {
)
);
indexRandom(true, builders);
+ indexRandomForMultipleSlices("idx");
ensureSearchable();
}
@@ -354,6 +355,7 @@ public void testNestedAsSubAggregation() throws Exception {
}
public void testNestNestedAggs() throws Exception {
+ indexRandomForConcurrentSearch("idx_nested_nested_aggs");
SearchResponse response = client().prepareSearch("idx_nested_nested_aggs")
.addAggregation(
nested("level1", "nested1").subAggregation(
@@ -607,6 +609,7 @@ public void testNestedSameDocIdProcessedMultipleTime() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("idx4");
SearchResponse response = client().prepareSearch("idx4")
.addAggregation(
@@ -782,6 +785,7 @@ public void testFilterAggInsideNestedAgg() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("classes");
SearchResponse response = client().prepareSearch("classes")
.addAggregation(
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/RangeIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/RangeIT.java
index 64ab6f1382ac3..5812b7796c33e 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/RangeIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/RangeIT.java
@@ -184,6 +184,7 @@ public void setupSuiteScopeCluster() throws Exception {
builders.add(client().prepareIndex("new_index").setSource(Collections.emptyMap()));
indexRandom(true, builders);
+ indexRandomForMultipleSlices("idx", "old_index", "new_index");
ensureSearchable();
}
@@ -917,6 +918,7 @@ public void testOverlappingRanges() throws Exception {
}
public void testEmptyAggregation() throws Exception {
+ indexRandomForConcurrentSearch("empty_bucket_idx");
SearchResponse searchResponse = client().prepareSearch("empty_bucket_idx")
.setQuery(matchAllQuery())
.addAggregation(
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SamplerIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SamplerIT.java
index 7033c42c5d661..c7b03d21cb6bb 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SamplerIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SamplerIT.java
@@ -34,9 +34,9 @@
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import org.opensearch.action.admin.indices.refresh.RefreshRequest;
import org.opensearch.action.search.SearchResponse;
import org.opensearch.action.search.SearchType;
+import org.opensearch.action.support.WriteRequest;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.FeatureFlags;
import org.opensearch.index.query.TermQueryBuilder;
@@ -132,13 +132,14 @@ public void setupSuiteScopeCluster() throws Exception {
client().prepareIndex("test")
.setId("" + i)
.setSource("author", parts[5], "name", parts[2], "genre", parts[8], "price", Float.parseFloat(parts[3]))
+ .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
client().prepareIndex("idx_unmapped_author")
.setId("" + i)
.setSource("name", parts[2], "genre", parts[8], "price", Float.parseFloat(parts[3]))
+ .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
}
- client().admin().indices().refresh(new RefreshRequest("test")).get();
}
public void testIssue10719() throws Exception {
@@ -195,6 +196,23 @@ public void testSimpleSampler() throws Exception {
assertThat(maxBooksPerAuthor, equalTo(3L));
}
+ public void testSimpleSamplerShardSize() throws Exception {
+ final int SHARD_SIZE = randomIntBetween(1, 3);
+ SamplerAggregationBuilder sampleAgg = sampler("sample").shardSize(SHARD_SIZE);
+ sampleAgg.subAggregation(terms("authors").field("author"));
+ SearchResponse response = client().prepareSearch("test")
+ .setSearchType(SearchType.QUERY_THEN_FETCH)
+ .setQuery(new TermQueryBuilder("genre", "fantasy"))
+ .setFrom(0)
+ .setSize(60)
+ .addAggregation(sampleAgg)
+ .get();
+ assertSearchResponse(response);
+ Sampler sample = response.getAggregations().get("sample");
+ Terms authors = sample.getAggregations().get("authors");
+ assertEquals(SHARD_SIZE * NUM_SHARDS, sample.getDocCount());
+ }
+
public void testUnmappedChildAggNoDiversity() throws Exception {
SamplerAggregationBuilder sampleAgg = sampler("sample").shardSize(100);
sampleAgg.subAggregation(terms("authors").field("author"));
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/CardinalityWithRequestBreakerIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/CardinalityWithRequestBreakerIT.java
index 2bf5230c67b43..85c36ec0ba78d 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/CardinalityWithRequestBreakerIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/CardinalityWithRequestBreakerIT.java
@@ -76,6 +76,7 @@ protected Settings featureFlagSettings() {
/**
* Test that searches using cardinality aggregations returns all request breaker memory.
*/
+ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/10154")
public void testRequestBreaker() throws Exception {
final String requestBreaker = randomIntBetween(1, 10000) + "kb";
logger.info("--> Using request breaker setting: {}", requestBreaker);
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java
index 18484c8a60ed7..41bbffc13658b 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java
@@ -693,6 +693,7 @@ public void testAvgMovavgDerivNPE() throws Exception {
}
refresh();
+ indexRandomForConcurrentSearch("movavg_npe");
SearchResponse response = client().prepareSearch("movavg_npe")
.addAggregation(
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MaxBucketIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MaxBucketIT.java
index bb90c1294ecb8..dc3b690c7f78f 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MaxBucketIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MaxBucketIT.java
@@ -544,13 +544,13 @@ public void testNested() throws Exception {
/**
* https://github.com/elastic/elasticsearch/issues/33514
- *
+ *
* This bug manifests as the max_bucket agg ("peak") being added to the response twice, because
* the pipeline agg is run twice. This makes invalid JSON and breaks conversion to maps.
* The bug was caused by an UnmappedTerms being the chosen as the first reduction target. UnmappedTerms
* delegated reduction to the first non-unmapped agg, which would reduce and run pipeline aggs. But then
* execution returns to the UnmappedTerms and _it_ runs pipelines as well, doubling up on the values.
- *
+ *
* Applies to any pipeline agg, not just max.
*/
public void testFieldIsntWrittenOutTwice() throws Exception {
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MovAvgIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MovAvgIT.java
index 0cf89778c6e99..8ad3107ac33ac 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MovAvgIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/MovAvgIT.java
@@ -1168,7 +1168,7 @@ public void testHoltWintersMinimization() {
* the default settings. Which means our mock histo will match the generated result (which it won't
* if the minimizer is actually working, since the coefficients will be different and thus generate different
* data)
- *
+ *
* We can simulate this by setting the window size == size of histo
*/
public void testMinimizeNotEnoughData() {
@@ -1320,6 +1320,7 @@ public void testPredictWithNonEmptyBuckets() throws Exception {
.setSource(jsonBuilder().startObject().field(INTERVAL_FIELD, i).field(VALUE_FIELD2, 10).endObject())
);
}
+ indexRandomForConcurrentSearch("predict_non_empty");
bulkBuilder.get();
ensureSearchable();
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchRedStateIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchRedStateIndexIT.java
index 44c4981dfdb36..bd623ccdf2731 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchRedStateIndexIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchRedStateIndexIT.java
@@ -153,6 +153,7 @@ private void buildRedIndex(int numShards) throws Exception {
client().prepareIndex("test").setId("" + i).setSource("field1", "value1").get();
}
refresh();
+ indexRandomForConcurrentSearch("test");
internalCluster().stopRandomDataNode();
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWhileCreatingIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWhileCreatingIndexIT.java
index 71af7215c4eb7..a5989b693d332 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWhileCreatingIndexIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWhileCreatingIndexIT.java
@@ -106,6 +106,7 @@ private void searchWhileCreatingIndex(boolean createIndex, int numberOfReplicas)
}
client().prepareIndex("test").setId(id).setSource("field", "test").get();
RefreshResponse refreshResponse = client().admin().indices().prepareRefresh("test").get();
+ indexRandomForConcurrentSearch("test");
// at least one shard should be successful when refreshing
assertThat(refreshResponse.getSuccessfulShards(), greaterThanOrEqualTo(1));
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/basic/TransportTwoNodesSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/basic/TransportTwoNodesSearchIT.java
index ce5f3f63faa66..edceb0cbc0d24 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/basic/TransportTwoNodesSearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/basic/TransportTwoNodesSearchIT.java
@@ -128,6 +128,7 @@ private Set prepareData(int numShards) throws Exception {
fullExpectedIds.add(Integer.toString(i));
}
refresh();
+ indexRandomForConcurrentSearch("test");
return fullExpectedIds;
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fetch/FetchSubPhasePluginIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fetch/FetchSubPhasePluginIT.java
index 86df25c4dad65..87f2153eb800f 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/fetch/FetchSubPhasePluginIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/fetch/FetchSubPhasePluginIT.java
@@ -119,6 +119,7 @@ public void testPlugin() throws Exception {
.actionGet();
client().admin().indices().prepareRefresh().get();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch()
.setSource(new SearchSourceBuilder().ext(Collections.singletonList(new TermVectorsFetchBuilder("test"))))
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/InnerHitsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/InnerHitsIT.java
index 9b3e1337418cc..1a730c01e4890 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/InnerHitsIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/InnerHitsIT.java
@@ -897,6 +897,7 @@ public void testNestedSource() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("index1");
// the field name (comments.message) used for source filtering should be the same as when using that field for
// other features (like in the query dsl or aggs) in order for consistency:
@@ -973,6 +974,7 @@ public void testInnerHitsWithIgnoreUnmapped() throws Exception {
client().prepareIndex("index1").setId("1").setSource("nested_type", Collections.singletonMap("key", "value")).get();
client().prepareIndex("index2").setId("3").setSource("key", "value").get();
refresh();
+ indexRandomForConcurrentSearch("index1", "index2");
SearchResponse response = client().prepareSearch("index1", "index2")
.setQuery(
@@ -1002,6 +1004,7 @@ public void testUseMaxDocInsteadOfSize() throws Exception {
.setRefreshPolicy(IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("index2");
QueryBuilder query = nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg).innerHit(
new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1)
);
@@ -1019,6 +1022,7 @@ public void testTooHighResultWindow() throws Exception {
)
.setRefreshPolicy(IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("index2");
SearchResponse response = client().prepareSearch("index2")
.setQuery(
nestedQuery("nested", matchQuery("nested.field", "value1"), ScoreMode.Avg).innerHit(
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/MatchedQueriesIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/MatchedQueriesIT.java
index 23b5d0cab0697..83cedb8c20e1d 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/MatchedQueriesIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/MatchedQueriesIT.java
@@ -91,6 +91,7 @@ public void testSimpleMatchedQueryFromFilteredQuery() throws Exception {
client().prepareIndex("test").setId("2").setSource("name", "test2", "number", 2).get();
client().prepareIndex("test").setId("3").setSource("name", "test3", "number", 3).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(
@@ -141,6 +142,7 @@ public void testSimpleMatchedQueryFromTopLevelFilter() throws Exception {
client().prepareIndex("test").setId("2").setSource("name", "test").get();
client().prepareIndex("test").setId("3").setSource("name", "test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(matchAllQuery())
@@ -192,6 +194,7 @@ public void testSimpleMatchedQueryFromTopLevelFilterAndFilteredQuery() throws Ex
client().prepareIndex("test").setId("2").setSource("name", "test", "title", "title2").get();
client().prepareIndex("test").setId("3").setSource("name", "test", "title", "title3").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(boolQuery().must(matchAllQuery()).filter(termsQuery("title", "title1", "title2", "title3").queryName("title")))
@@ -224,12 +227,13 @@ public void testSimpleMatchedQueryFromTopLevelFilterAndFilteredQuery() throws Ex
}
}
- public void testRegExpQuerySupportsName() {
+ public void testRegExpQuerySupportsName() throws InterruptedException {
createIndex("test1");
ensureGreen();
client().prepareIndex("test1").setId("1").setSource("title", "title1").get();
refresh();
+ indexRandomForConcurrentSearch("test1");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(QueryBuilders.regexpQuery("title", "title1").queryName("regex"))
@@ -246,12 +250,13 @@ public void testRegExpQuerySupportsName() {
}
}
- public void testPrefixQuerySupportsName() {
+ public void testPrefixQuerySupportsName() throws InterruptedException {
createIndex("test1");
ensureGreen();
client().prepareIndex("test1").setId("1").setSource("title", "title1").get();
refresh();
+ indexRandomForConcurrentSearch("test1");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(QueryBuilders.prefixQuery("title", "title").queryName("prefix"))
@@ -268,12 +273,13 @@ public void testPrefixQuerySupportsName() {
}
}
- public void testFuzzyQuerySupportsName() {
+ public void testFuzzyQuerySupportsName() throws InterruptedException {
createIndex("test1");
ensureGreen();
client().prepareIndex("test1").setId("1").setSource("title", "title1").get();
refresh();
+ indexRandomForConcurrentSearch("test1");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(QueryBuilders.fuzzyQuery("title", "titel1").queryName("fuzzy"))
@@ -290,12 +296,13 @@ public void testFuzzyQuerySupportsName() {
}
}
- public void testWildcardQuerySupportsName() {
+ public void testWildcardQuerySupportsName() throws InterruptedException {
createIndex("test1");
ensureGreen();
client().prepareIndex("test1").setId("1").setSource("title", "title1").get();
refresh();
+ indexRandomForConcurrentSearch("test1");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(QueryBuilders.wildcardQuery("title", "titl*").queryName("wildcard"))
@@ -312,12 +319,13 @@ public void testWildcardQuerySupportsName() {
}
}
- public void testSpanFirstQuerySupportsName() {
+ public void testSpanFirstQuerySupportsName() throws InterruptedException {
createIndex("test1");
ensureGreen();
client().prepareIndex("test1").setId("1").setSource("title", "title1 title2").get();
refresh();
+ indexRandomForConcurrentSearch("test1");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(QueryBuilders.spanFirstQuery(QueryBuilders.spanTermQuery("title", "title1"), 10).queryName("span"))
@@ -344,6 +352,7 @@ public void testMatchedWithShould() throws Exception {
client().prepareIndex("test").setId("1").setSource("content", "Lorem ipsum dolor sit amet").get();
client().prepareIndex("test").setId("2").setSource("content", "consectetur adipisicing elit").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// Execute search at least two times to load it in cache
int iter = scaledRandomIntBetween(2, 10);
@@ -378,6 +387,7 @@ public void testMatchedWithWrapperQuery() throws Exception {
client().prepareIndex("test").setId("1").setSource("content", "Lorem ipsum dolor sit amet").get();
refresh();
+ indexRandomForConcurrentSearch("test");
MatchQueryBuilder matchQueryBuilder = matchQuery("content", "amet").queryName("abc");
BytesReference matchBytes = XContentHelper.toXContent(matchQueryBuilder, MediaTypeRegistry.JSON, false);
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
index 4cdf5ae8e674f..2afa911223074 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
@@ -155,7 +155,7 @@ protected Collection> nodePlugins() {
return Arrays.asList(InternalSettingsPlugin.class, MockKeywordPlugin.class, MockAnalysisPlugin.class);
}
- public void testHighlightingWithKeywordIgnoreBoundaryScanner() throws IOException {
+ public void testHighlightingWithKeywordIgnoreBoundaryScanner() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder();
mappings.startObject();
mappings.startObject("properties")
@@ -177,6 +177,7 @@ public void testHighlightingWithKeywordIgnoreBoundaryScanner() throws IOExceptio
.setSource(jsonBuilder().startObject().array("tags", "foo baz", "foo baz", "foo baz", "foo bar").field("sort", 2).endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
for (BoundaryScannerType scanner : BoundaryScannerType.values()) {
SearchResponse search = client().prepareSearch()
@@ -190,12 +191,13 @@ public void testHighlightingWithKeywordIgnoreBoundaryScanner() throws IOExceptio
}
}
- public void testHighlightingWithStoredKeyword() throws IOException {
+ public void testHighlightingWithStoredKeyword() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder();
mappings.startObject();
mappings.startObject("properties").startObject("text").field("type", "keyword").field("store", true).endObject().endObject();
mappings.endObject();
assertAcked(prepareCreate("test").setMapping(mappings));
+ indexRandomForConcurrentSearch("test");
client().prepareIndex("test").setId("1").setSource(jsonBuilder().startObject().field("text", "foo").endObject()).get();
refresh();
SearchResponse search = client().prepareSearch()
@@ -205,7 +207,7 @@ public void testHighlightingWithStoredKeyword() throws IOException {
assertHighlight(search, 0, "text", 0, equalTo("foo "));
}
- public void testHighlightingWithWildcardName() throws IOException {
+ public void testHighlightingWithWildcardName() throws IOException, InterruptedException {
// test the kibana case with * as fieldname that will try highlight all fields including meta fields
XContentBuilder mappings = jsonBuilder();
mappings.startObject();
@@ -221,6 +223,7 @@ public void testHighlightingWithWildcardName() throws IOException {
assertAcked(prepareCreate("test").setMapping(mappings));
client().prepareIndex("test").setId("1").setSource(jsonBuilder().startObject().field("text", "text").endObject()).get();
refresh();
+ indexRandomForConcurrentSearch("test");
for (String type : ALL_TYPES) {
SearchResponse search = client().prepareSearch()
.setQuery(constantScoreQuery(matchQuery("text", "text")))
@@ -230,7 +233,7 @@ public void testHighlightingWithWildcardName() throws IOException {
}
}
- public void testFieldAlias() throws IOException {
+ public void testFieldAlias() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder().startObject()
.startObject("properties")
.startObject("text")
@@ -248,7 +251,7 @@ public void testFieldAlias() throws IOException {
client().prepareIndex("test").setId("1").setSource("text", "foo").get();
refresh();
-
+ indexRandomForConcurrentSearch("test");
for (String type : ALL_TYPES) {
HighlightBuilder builder = new HighlightBuilder().field(new Field("alias").highlighterType(type))
.requireFieldMatch(randomBoolean());
@@ -257,7 +260,7 @@ public void testFieldAlias() throws IOException {
}
}
- public void testFieldAliasWithSourceLookup() throws IOException {
+ public void testFieldAliasWithSourceLookup() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder().startObject()
.startObject("properties")
.startObject("text")
@@ -276,7 +279,7 @@ public void testFieldAliasWithSourceLookup() throws IOException {
client().prepareIndex("test").setId("1").setSource("text", "foo bar").get();
refresh();
-
+ indexRandomForConcurrentSearch("test");
for (String type : ALL_TYPES) {
HighlightBuilder builder = new HighlightBuilder().field(new Field("alias").highlighterType(type))
.requireFieldMatch(randomBoolean());
@@ -285,7 +288,7 @@ public void testFieldAliasWithSourceLookup() throws IOException {
}
}
- public void testFieldAliasWithWildcardField() throws IOException {
+ public void testFieldAliasWithWildcardField() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder().startObject()
.startObject("properties")
.startObject("keyword")
@@ -301,13 +304,14 @@ public void testFieldAliasWithWildcardField() throws IOException {
client().prepareIndex("test").setId("1").setSource("keyword", "foo").get();
refresh();
+ indexRandomForConcurrentSearch("test");
HighlightBuilder builder = new HighlightBuilder().field(new Field("al*")).requireFieldMatch(false);
SearchResponse search = client().prepareSearch().setQuery(matchQuery("alias", "foo")).highlighter(builder).get();
assertHighlight(search, 0, "alias", 0, equalTo("foo "));
}
- public void testHighlightingWhenFieldsAreNotStoredThereIsNoSource() throws IOException {
+ public void testHighlightingWhenFieldsAreNotStoredThereIsNoSource() throws IOException, InterruptedException {
XContentBuilder mappings = jsonBuilder();
mappings.startObject();
mappings.startObject("_source")
@@ -334,6 +338,7 @@ public void testHighlightingWhenFieldsAreNotStoredThereIsNoSource() throws IOExc
.setSource(jsonBuilder().startObject().field("unstored_text", "text").field("text", "text").endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
for (String type : ALL_TYPES) {
SearchResponse search = client().prepareSearch()
.setQuery(constantScoreQuery(matchQuery("text", "text")))
@@ -350,7 +355,7 @@ public void testHighlightingWhenFieldsAreNotStoredThereIsNoSource() throws IOExc
}
// see #3486
- public void testHighTermFrequencyDoc() throws IOException {
+ public void testHighTermFrequencyDoc() throws IOException, InterruptedException {
assertAcked(prepareCreate("test").setMapping("name", "type=text,term_vector=with_positions_offsets,store=" + randomBoolean()));
StringBuilder builder = new StringBuilder();
for (int i = 0; i < 6000; i++) {
@@ -358,6 +363,7 @@ public void testHighTermFrequencyDoc() throws IOException {
}
client().prepareIndex("test").setId("1").setSource("name", builder.toString()).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse search = client().prepareSearch()
.setQuery(constantScoreQuery(matchQuery("name", "abc")))
.highlighter(new HighlightBuilder().field("name"))
@@ -385,6 +391,7 @@ public void testEnsureNoNegativeOffsets() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse search = client().prepareSearch()
.setQuery(matchQuery("long_term", "thisisaverylongwordandmakessurethisfails foo highlighed"))
.highlighter(new HighlightBuilder().field("long_term", 18, 1).highlighterType("fvh"))
@@ -671,7 +678,7 @@ public void testHighlightIssue1994() throws Exception {
assertHighlight(search, 0, "titleTV", 1, 2, equalTo("highlight other text"));
}
- public void testGlobalHighlightingSettingsOverriddenAtFieldLevel() {
+ public void testGlobalHighlightingSettingsOverriddenAtFieldLevel() throws InterruptedException {
createIndex("test");
ensureGreen();
@@ -684,6 +691,7 @@ public void testGlobalHighlightingSettingsOverriddenAtFieldLevel() {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1 and field2 produces different tags");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "test"))
@@ -734,6 +742,7 @@ public void testHighlightingOnWildcardFields() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field*");
SearchSourceBuilder source = searchSource()
@@ -783,6 +792,7 @@ public void testForceSourceWithSourceDisabled() throws Exception {
.setSource("field1", "The quick brown fox jumps over the lazy dog", "field2", "second field content")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
// works using stored field
SearchResponse searchResponse = client().prepareSearch("test")
@@ -823,6 +833,7 @@ public void testPlainHighlighter() throws Exception {
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "test"))
@@ -1025,6 +1036,7 @@ public void testFVHManyMatches() throws Exception {
String value = new String(new char[1024 * 256 / pattern.length()]).replace("\0", pattern);
client().prepareIndex("test").setSource("field1", value).get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1 with default phrase limit");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "t"))
@@ -1116,6 +1128,7 @@ private void checkMatchedFieldsCase(boolean requireFieldMatch) throws Exception
);
index("test", "type1", "3", "foo", "weird", "bar", "result");
refresh();
+ indexRandomForConcurrentSearch("test");
Field fooField = new Field("foo").numOfFragments(1)
.order("score")
@@ -1408,6 +1421,7 @@ public void testMultiMapperVectorWithStore() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource("title", "this is a test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse search = client().prepareSearch()
@@ -1453,6 +1467,7 @@ public void testMultiMapperVectorFromSource() throws Exception {
client().prepareIndex("test").setId("1").setSource("title", "this is a test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse search = client().prepareSearch()
@@ -1498,6 +1513,7 @@ public void testMultiMapperNoVectorWithStore() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource("title", "this is a test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse search = client().prepareSearch()
@@ -1542,6 +1558,7 @@ public void testMultiMapperNoVectorFromSource() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource("title", "this is a test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse search = client().prepareSearch()
@@ -1571,6 +1588,7 @@ public void testFastVectorHighlighterShouldFailIfNoTermVectors() throws Exceptio
.setSource("title", "This is a test for the enabling fast vector highlighter");
}
indexRandom(true, indexRequestBuilders);
+ indexRandomForConcurrentSearch("test");
SearchResponse search = client().prepareSearch()
.setQuery(matchPhraseQuery("title", "this is a test"))
@@ -1608,6 +1626,7 @@ public void testDisableFastVectorHighlighter() throws Exception {
.setSource("title", "This is a test for the workaround for the fast vector highlighting SOLR-3724");
}
indexRandom(true, indexRequestBuilders);
+ indexRandomForConcurrentSearch("test");
SearchResponse search = client().prepareSearch()
.setQuery(matchPhraseQuery("title", "test for the workaround"))
@@ -1669,6 +1688,7 @@ public void testFSHHighlightAllMvFragments() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("tags", "tag"))
@@ -1686,11 +1706,12 @@ public void testFSHHighlightAllMvFragments() throws Exception {
);
}
- public void testBoostingQuery() {
+ public void testBoostingQuery() throws InterruptedException {
createIndex("test");
ensureGreen();
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -1702,11 +1723,12 @@ public void testBoostingQuery() {
assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog"));
}
- public void testBoostingQueryTermVector() throws IOException {
+ public void testBoostingQueryTermVector() throws IOException, InterruptedException {
assertAcked(prepareCreate("test").setMapping(type1TermVectorMapping()));
ensureGreen();
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -1718,12 +1740,13 @@ public void testBoostingQueryTermVector() throws IOException {
assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog"));
}
- public void testCommonTermsQuery() {
+ public void testCommonTermsQuery() throws InterruptedException {
createIndex("test");
ensureGreen();
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(commonTermsQuery("field2", "quick brown").cutoffFrequency(100))
@@ -1733,12 +1756,13 @@ public void testCommonTermsQuery() {
assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog"));
}
- public void testCommonTermsTermVector() throws IOException {
+ public void testCommonTermsTermVector() throws IOException, InterruptedException {
assertAcked(prepareCreate("test").setMapping(type1TermVectorMapping()));
ensureGreen();
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(commonTermsQuery("field2", "quick brown").cutoffFrequency(100))
.highlighter(highlight().field("field2").order("score").preTags("").postTags(" "));
@@ -1764,6 +1788,7 @@ public void testPlainHighlightDifferentFragmenter() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchPhraseQuery("tags", "long tag"))
@@ -1816,12 +1841,13 @@ public void testPlainHighlightDifferentFragmenter() throws Exception {
);
}
- public void testPlainHighlighterMultipleFields() {
+ public void testPlainHighlighterMultipleFields() throws InterruptedException {
createIndex("test");
ensureGreen();
index("test", "type1", "1", "field1", "The quick brown fox", "field2", "The slow brown fox");
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("field1", "fox"))
@@ -1834,7 +1860,7 @@ public void testPlainHighlighterMultipleFields() {
assertHighlight(response, 0, "field2", 0, 1, equalTo("The slow brown <2>fox2>"));
}
- public void testFastVectorHighlighterMultipleFields() {
+ public void testFastVectorHighlighterMultipleFields() throws InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
"field1",
@@ -1847,6 +1873,7 @@ public void testFastVectorHighlighterMultipleFields() {
index("test", "type1", "1", "field1", "The quick brown fox", "field2", "The slow brown fox");
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("field1", "fox"))
@@ -1864,6 +1891,7 @@ public void testMissingStoredField() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource(jsonBuilder().startObject().field("field", "highlight").endObject()).get();
refresh();
+ indexRandomForConcurrentSearch("test");
// This query used to fail when the field to highlight was absent
SearchResponse response = client().prepareSearch("test")
@@ -1904,6 +1932,7 @@ public void testNumericHighlighting() throws Exception {
.setSource("text", "opensearch test", "byte", 25, "short", 42, "int", 100, "long", -1, "float", 3.2f, "double", 42.42)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("text", "test"))
@@ -1926,6 +1955,7 @@ public void testResetTwice() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource("text", "opensearch test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("text", "test"))
@@ -1935,7 +1965,7 @@ public void testResetTwice() throws Exception {
assertHitCount(response, 1L);
}
- public void testHighlightUsesHighlightQuery() throws IOException {
+ public void testHighlightUsesHighlightQuery() throws IOException, InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
"text",
@@ -1946,6 +1976,7 @@ public void testHighlightUsesHighlightQuery() throws IOException {
index("test", "type1", "1", "text", "Testing the highlight query feature");
refresh();
+ indexRandomForConcurrentSearch("test");
for (String type : ALL_TYPES) {
HighlightBuilder.Field field = new HighlightBuilder.Field("text");
@@ -1981,7 +2012,7 @@ private static String randomStoreField() {
return "";
}
- public void testHighlightNoMatchSize() throws IOException {
+ public void testHighlightNoMatchSize() throws IOException, InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
"text",
@@ -1993,6 +2024,7 @@ public void testHighlightNoMatchSize() throws IOException {
String text = "I am pretty long so some of me should get cut off. Second sentence";
index("test", "type1", "1", "text", text);
refresh();
+ indexRandomForConcurrentSearch("test");
// When you don't set noMatchSize you don't get any results if there isn't anything to highlight.
HighlightBuilder.Field field = new HighlightBuilder.Field("text").fragmentSize(21).numOfFragments(1).highlighterType("plain");
@@ -2091,7 +2123,7 @@ public void testHighlightNoMatchSize() throws IOException {
assertNotHighlighted(response, 0, "text");
}
- public void testHighlightNoMatchSizeWithMultivaluedFields() throws IOException {
+ public void testHighlightNoMatchSizeWithMultivaluedFields() throws IOException, InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
"text",
@@ -2104,6 +2136,7 @@ public void testHighlightNoMatchSizeWithMultivaluedFields() throws IOException {
String text2 = "I am short";
index("test", "type1", "1", "text", new String[] { text1, text2 });
refresh();
+ indexRandomForConcurrentSearch("test");
// The no match fragment should come from the first value of a multi-valued field
HighlightBuilder.Field field = new HighlightBuilder.Field("text").fragmentSize(21)
@@ -2186,7 +2219,7 @@ public void testHighlightNoMatchSizeWithMultivaluedFields() throws IOException {
assertNotHighlighted(response, 0, "text");
}
- public void testHighlightNoMatchSizeNumberOfFragments() throws IOException {
+ public void testHighlightNoMatchSizeNumberOfFragments() throws IOException, InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
"text",
@@ -2200,6 +2233,7 @@ public void testHighlightNoMatchSizeNumberOfFragments() throws IOException {
String text3 = "This is the fifth sentence";
index("test", "type1", "1", "text", new String[] { text1, text2, text3 });
refresh();
+ indexRandomForConcurrentSearch("test");
// The no match fragment should come from the first value of a multi-valued field
HighlightBuilder.Field field = new HighlightBuilder.Field("text").fragmentSize(1)
@@ -2243,6 +2277,7 @@ public void testPostingsHighlighter() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy quick dog")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "test"))
@@ -2320,6 +2355,7 @@ public void testPostingsHighlighterMultipleFields() throws Exception {
"The slow brown fox. Second sentence."
);
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(QueryBuilders.matchQuery("field1", "fox"))
@@ -2344,6 +2380,7 @@ public void testPostingsHighlighterNumberOfFragments() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "fox"))
@@ -2376,6 +2413,7 @@ public void testPostingsHighlighterNumberOfFragments() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
source = searchSource().query(termQuery("field1", "fox"))
.highlighter(highlight().field(new Field("field1").numOfFragments(0).preTags("").postTags(" ")));
@@ -2412,7 +2450,7 @@ public void testPostingsHighlighterNumberOfFragments() throws Exception {
}
}
- public void testMultiMatchQueryHighlight() throws IOException {
+ public void testMultiMatchQueryHighlight() throws IOException, InterruptedException {
XContentBuilder mapping = XContentFactory.jsonBuilder()
.startObject()
.startObject("properties")
@@ -2434,6 +2472,7 @@ public void testMultiMatchQueryHighlight() throws IOException {
.setSource("field1", "The quick brown fox jumps over", "field2", "The quick brown fox jumps over")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
final int iters = scaledRandomIntBetween(20, 30);
for (int i = 0; i < iters; i++) {
String highlighterType = rarely() ? null : RandomPicks.randomFrom(random(), ALL_TYPES);
@@ -2479,6 +2518,7 @@ public void testPostingsHighlighterOrderByScore() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(termQuery("field1", "sentence"))
@@ -2565,6 +2605,7 @@ public void testPostingsHighlighterMultiMapperWithStore() throws Exception {
ensureGreen();
client().prepareIndex("test").setId("1").setSource("title", "this is a test . Second sentence.").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse searchResponse = client().prepareSearch()
@@ -2623,6 +2664,7 @@ public void testPostingsHighlighterMultiMapperFromSource() throws Exception {
client().prepareIndex("test").setId("1").setSource("title", "this is a test").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// simple search on body with standard analyzer with a simple field query
SearchResponse searchResponse = client().prepareSearch()
@@ -2672,13 +2714,14 @@ public void testPostingsHighlighterShouldFailIfNoOffsets() throws Exception {
assertNoFailures(search);
}
- public void testPostingsHighlighterBoostingQuery() throws IOException {
+ public void testPostingsHighlighterBoostingQuery() throws IOException, InterruptedException {
assertAcked(prepareCreate("test").setMapping(type1PostingsffsetsMapping()));
ensureGreen();
client().prepareIndex("test")
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -2689,7 +2732,7 @@ public void testPostingsHighlighterBoostingQuery() throws IOException {
assertHighlight(searchResponse, 0, "field2", 0, 1, equalTo("The quick brown fox jumps over the lazy dog! Second sentence."));
}
- public void testPostingsHighlighterCommonTermsQuery() throws IOException {
+ public void testPostingsHighlighterCommonTermsQuery() throws IOException, InterruptedException {
assertAcked(prepareCreate("test").setMapping(type1PostingsffsetsMapping()));
ensureGreen();
@@ -2697,6 +2740,7 @@ public void testPostingsHighlighterCommonTermsQuery() throws IOException {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(commonTermsQuery("field2", "quick brown").cutoffFrequency(100))
@@ -2738,6 +2782,7 @@ public void testPostingsHighlighterPrefixQuery() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(prefixQuery("field2", "qui")).highlighter(highlight().field("field2"));
@@ -2760,6 +2805,7 @@ public void testPostingsHighlighterFuzzyQuery() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(fuzzyQuery("field2", "quck")).highlighter(highlight().field("field2"));
@@ -2783,6 +2829,7 @@ public void testPostingsHighlighterRegexpQuery() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(regexpQuery("field2", "qu[a-l]+k")).highlighter(highlight().field("field2"));
@@ -2806,6 +2853,7 @@ public void testPostingsHighlighterWildcardQuery() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(wildcardQuery("field2", "qui*")).highlighter(highlight().field("field2"));
@@ -2840,6 +2888,7 @@ public void testPostingsHighlighterTermRangeQuery() throws Exception {
client().prepareIndex("test").setSource("field1", "this is a test", "field2", "aaab").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(rangeQuery("field2").gte("aaaa").lt("zzzz"))
@@ -2857,6 +2906,7 @@ public void testPostingsHighlighterQueryString() throws Exception {
.setSource("field1", "this is a test", "field2", "The quick brown fox jumps over the lazy dog! Second sentence.")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field2");
SearchSourceBuilder source = searchSource().query(queryStringQuery("qui*").defaultField("field2"))
@@ -2878,6 +2928,7 @@ public void testPostingsHighlighterRegexpQueryWithinConstantScoreQuery() throws
client().prepareIndex("test").setSource("field1", "The photography word will get highlighted").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(constantScoreQuery(regexpQuery("field1", "pho[a-z]+")))
@@ -2892,6 +2943,7 @@ public void testPostingsHighlighterMultiTermQueryMultipleLevels() throws Excepti
client().prepareIndex("test").setSource("field1", "The photography word will get highlighted").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -2909,6 +2961,7 @@ public void testPostingsHighlighterPrefixQueryWithinBooleanQuery() throws Except
client().prepareIndex("test").setSource("field1", "The photography word will get highlighted").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -2924,6 +2977,7 @@ public void testPostingsHighlighterQueryStringWithinFilteredQuery() throws Excep
client().prepareIndex("test").setSource("field1", "The photography word will get highlighted").get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("--> highlighting and searching on field1");
SearchSourceBuilder source = searchSource().query(
@@ -3028,7 +3082,7 @@ public void testFastVectorHighlighterPhraseBoost() throws Exception {
* because it doesn't support the concept of terms having a different weight based on position.
* @param highlighterType highlighter to test
*/
- private void phraseBoostTestCase(String highlighterType) {
+ private void phraseBoostTestCase(String highlighterType) throws InterruptedException {
ensureGreen();
StringBuilder text = new StringBuilder();
text.append("words words junk junk junk junk junk junk junk junk highlight junk junk junk junk together junk\n");
@@ -3041,6 +3095,7 @@ private void phraseBoostTestCase(String highlighterType) {
}
index("test", "type1", "1", "field1", text.toString());
refresh();
+ indexRandomForConcurrentSearch("test");
// Match queries
phraseBoostTestCaseForClauses(
@@ -3109,7 +3164,7 @@ private > void phraseBoostTestCaseForClauses(
assertHighlight(response, 0, "field1", 0, 1, highlightedMatcher);
}
- public void testGeoFieldHighlightingWithDifferentHighlighters() throws IOException {
+ public void testGeoFieldHighlightingWithDifferentHighlighters() throws IOException, InterruptedException {
// check that we do not get an exception for geo_point fields in case someone tries to highlight
// it accidentially with a wildcard
// see https://github.com/elastic/elasticsearch/issues/17537
@@ -3133,6 +3188,7 @@ public void testGeoFieldHighlightingWithDifferentHighlighters() throws IOExcepti
.setSource(jsonBuilder().startObject().field("text", "Arbitrary text field which will should not cause a failure").endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
String highlighterType = randomFrom(ALL_TYPES);
QueryBuilder query = QueryBuilders.boolQuery()
.should(
@@ -3150,7 +3206,7 @@ public void testGeoFieldHighlightingWithDifferentHighlighters() throws IOExcepti
assertThat(search.getHits().getAt(0).getHighlightFields().get("text").fragments().length, equalTo(1));
}
- public void testGeoFieldHighlightingWhenQueryGetsRewritten() throws IOException {
+ public void testGeoFieldHighlightingWhenQueryGetsRewritten() throws IOException, InterruptedException {
// same as above but in this example the query gets rewritten during highlighting
// see https://github.com/elastic/elasticsearch/issues/17537#issuecomment-244939633
XContentBuilder mappings = jsonBuilder();
@@ -3177,6 +3233,7 @@ public void testGeoFieldHighlightingWhenQueryGetsRewritten() throws IOException
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
QueryBuilder query = QueryBuilders.functionScoreQuery(
QueryBuilders.boolQuery()
@@ -3192,7 +3249,7 @@ public void testGeoFieldHighlightingWhenQueryGetsRewritten() throws IOException
assertThat(search.getHits().getTotalHits().value, equalTo(1L));
}
- public void testKeywordFieldHighlighting() throws IOException {
+ public void testKeywordFieldHighlighting() throws IOException, InterruptedException {
// check that keyword highlighting works
XContentBuilder mappings = jsonBuilder();
mappings.startObject();
@@ -3205,6 +3262,7 @@ public void testKeywordFieldHighlighting() throws IOException {
.setSource(jsonBuilder().startObject().field("keyword_field", "some text").endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse search = client().prepareSearch()
.setSource(
new SearchSourceBuilder().query(QueryBuilders.matchQuery("keyword_field", "some text"))
@@ -3238,6 +3296,7 @@ public void testCopyToFields() throws Exception {
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch()
.setQuery(matchQuery("foo_copy", "brown"))
.highlighter(new HighlightBuilder().field(new Field("foo_copy")))
@@ -3287,7 +3346,7 @@ public void testACopyFieldWithNestedQuery() throws Exception {
)
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
-
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(nestedQuery("foo", matchQuery("foo.text", "brown cow"), ScoreMode.None))
.highlighter(new HighlightBuilder().field(new Field("foo_text").highlighterType("fvh")).requireFieldMatch(false))
@@ -3305,6 +3364,7 @@ public void testFunctionScoreQueryHighlight() throws Exception {
.setSource(jsonBuilder().startObject().field("text", "brown").endObject())
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(new FunctionScoreQueryBuilder(QueryBuilders.prefixQuery("text", "bro")))
@@ -3322,6 +3382,7 @@ public void testFiltersFunctionScoreQueryHighlight() throws Exception {
.setSource(jsonBuilder().startObject().field("text", "brown").field("enable", "yes").endObject())
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
FunctionScoreQueryBuilder.FilterFunctionBuilder filterBuilder = new FunctionScoreQueryBuilder.FilterFunctionBuilder(
QueryBuilders.termQuery("enable", "yes"),
new RandomScoreFunctionBuilder()
@@ -3416,6 +3477,7 @@ public void testWithNestedQuery() throws Exception {
)
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
for (String type : new String[] { "unified", "plain" }) {
SearchResponse searchResponse = client().prepareSearch()
@@ -3473,6 +3535,7 @@ public void testWithNormalizer() throws Exception {
.setSource("keyword", "Hello World")
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
for (String highlighterType : new String[] { "unified", "plain" }) {
SearchResponse searchResponse = client().prepareSearch()
@@ -3495,6 +3558,7 @@ public void testDisableHighlightIdField() throws Exception {
.setSource("keyword", "Hello World")
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
for (String highlighterType : new String[] { "plain", "unified" }) {
SearchResponse searchResponse = client().prepareSearch()
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/functionscore/RandomScoreFunctionIT.java b/server/src/internalClusterTest/java/org/opensearch/search/functionscore/RandomScoreFunctionIT.java
index 8f43cefd2d53b..69e30fc879dd8 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/functionscore/RandomScoreFunctionIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/functionscore/RandomScoreFunctionIT.java
@@ -135,6 +135,7 @@ public void testConsistentHitsWithSameSeed() throws Exception {
}
flush();
refresh();
+ indexRandomForConcurrentSearch("test");
int outerIters = scaledRandomIntBetween(10, 20);
for (int o = 0; o < outerIters; o++) {
final int seed = randomInt();
@@ -207,6 +208,7 @@ public void testScoreAccessWithinScript() throws Exception {
.get();
}
refresh();
+ indexRandomForConcurrentSearch("test");
Map params = new HashMap<>();
params.put("factor", randomIntBetween(2, 4));
@@ -298,6 +300,7 @@ public void testSeedReportedInExplain() throws Exception {
index("test", "type", "1", jsonBuilder().startObject().endObject());
flush();
refresh();
+ indexRandomForConcurrentSearch("test");
int seed = 12345678;
@@ -317,6 +320,7 @@ public void testSeedAndNameReportedInExplain() throws Exception {
index("test", "type", "1", jsonBuilder().startObject().endObject());
flush();
refresh();
+ indexRandomForConcurrentSearch("test");
int seed = 12345678;
@@ -368,6 +372,7 @@ public void testScoreRange() throws Exception {
}
flush();
refresh();
+ indexRandomForConcurrentSearch("test");
int iters = scaledRandomIntBetween(10, 20);
for (int i = 0; i < iters; ++i) {
SearchResponse searchResponse = client().prepareSearch()
@@ -390,6 +395,7 @@ public void testSeeds() throws Exception {
index("test", "type", "" + i, jsonBuilder().startObject().endObject());
}
flushAndRefresh();
+ indexRandomForConcurrentSearch("test");
assertNoFailures(
client().prepareSearch()
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java
index 00524c6e04707..ba519be04edff 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java
@@ -266,6 +266,7 @@ public void testShapeRelations() throws Exception {
client().prepareIndex("shapes").setId("1").setSource(data, MediaTypeRegistry.JSON).get();
client().admin().indices().prepareRefresh().get();
+ indexRandomForConcurrentSearch("shapes");
// Point in polygon
SearchResponse result = client().prepareSearch()
@@ -427,6 +428,7 @@ public void testBulk() throws Exception {
client().admin().indices().prepareCreate("countries").setSettings(settings).setMapping(xContentBuilder).get();
BulkResponse bulk = client().prepareBulk().add(bulkAction, 0, bulkAction.length, null, xContentBuilder.contentType()).get();
+ indexRandomForConcurrentSearch("countries");
for (BulkItemResponse item : bulk.getItems()) {
assertFalse("unable to index data", item.isFailed());
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java b/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java
index 87435bb0bd09d..dc7c4e687c2fa 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java
@@ -125,6 +125,7 @@ public void testSimpleMoreLikeThis() throws Exception {
client().index(indexRequest("test").id("2").source(jsonBuilder().startObject().field("text", "lucene release").endObject()))
.actionGet();
client().admin().indices().refresh(refreshRequest()).actionGet();
+ indexRandomForConcurrentSearch("test");
logger.info("Running moreLikeThis");
SearchResponse response = client().prepareSearch()
@@ -155,6 +156,7 @@ public void testSimpleMoreLikeThisWithTypes() throws Exception {
client().index(indexRequest("test").id("2").source(jsonBuilder().startObject().field("text", "lucene release").endObject()))
.actionGet();
client().admin().indices().refresh(refreshRequest()).actionGet();
+ indexRandomForConcurrentSearch("test");
logger.info("Running moreLikeThis");
SearchResponse response = client().prepareSearch()
@@ -190,6 +192,7 @@ public void testMoreLikeThisForZeroTokensInOneOfTheAnalyzedFields() throws Excep
).actionGet();
client().admin().indices().refresh(refreshRequest()).actionGet();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(
@@ -256,6 +259,7 @@ public void testMoreLikeThisWithAliases() throws Exception {
client().index(indexRequest("test").id("4").source(jsonBuilder().startObject().field("text", "opensearch release").endObject()))
.actionGet();
client().admin().indices().refresh(refreshRequest()).actionGet();
+ indexRandomForConcurrentSearch("test");
logger.info("Running moreLikeThis on index");
SearchResponse response = client().prepareSearch()
@@ -304,6 +308,7 @@ public void testMoreLikeThisWithAliasesInLikeDocuments() throws Exception {
client().index(indexRequest(indexName).id("3").source(jsonBuilder().startObject().field("text", "opensearch index").endObject()))
.actionGet();
refresh(indexName);
+ indexRandomForConcurrentSearch(indexName);
SearchResponse response = client().prepareSearch()
.setQuery(new MoreLikeThisQueryBuilder(null, new Item[] { new Item(aliasName, "1") }).minTermFreq(1).minDocFreq(1))
@@ -321,6 +326,7 @@ public void testMoreLikeThisIssue2197() throws Exception {
.get();
client().admin().indices().prepareRefresh("foo").get();
assertThat(ensureGreen(), equalTo(ClusterHealthStatus.GREEN));
+ indexRandomForConcurrentSearch("foo");
SearchResponse response = client().prepareSearch()
.setQuery(new MoreLikeThisQueryBuilder(null, new Item[] { new Item("foo", "1") }))
@@ -344,6 +350,7 @@ public void testMoreLikeWithCustomRouting() throws Exception {
.setRouting("2")
.get();
client().admin().indices().prepareRefresh("foo").get();
+ indexRandomForConcurrentSearch("foo");
SearchResponse response = client().prepareSearch()
.setQuery(new MoreLikeThisQueryBuilder(null, new Item[] { new Item("foo", "1").routing("2") }))
@@ -368,6 +375,7 @@ public void testMoreLikeThisIssueRoutingNotSerialized() throws Exception {
.setRouting("4000")
.get();
client().admin().indices().prepareRefresh("foo").get();
+ indexRandomForConcurrentSearch("foo");
SearchResponse response = client().prepareSearch()
.setQuery(new MoreLikeThisQueryBuilder(null, new Item[] { new Item("foo", "1").routing("4000") }))
.get();
@@ -401,6 +409,7 @@ public void testNumericField() throws Exception {
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
// Implicit list of fields -> ignore numeric fields
SearchResponse searchResponse = client().prepareSearch()
@@ -508,6 +517,7 @@ public void testMoreLikeThisWithFieldAlias() throws Exception {
index("test", "_doc", "1", "text", "lucene");
index("test", "_doc", "2", "text", "lucene release");
refresh();
+ indexRandomForConcurrentSearch("test");
Item item = new Item("test", "1");
QueryBuilder query = QueryBuilders.moreLikeThisQuery(new String[] { "alias" }, null, new Item[] { item })
@@ -548,6 +558,7 @@ public void testSimpleMoreLikeInclude() throws Exception {
.source(jsonBuilder().startObject().field("text", "Lucene has been ported to other programming languages").endObject())
).actionGet();
client().admin().indices().refresh(refreshRequest()).actionGet();
+ indexRandomForConcurrentSearch("test");
logger.info("Running More Like This with include true");
SearchResponse response = client().prepareSearch()
@@ -832,11 +843,12 @@ public void testSelectFields() throws IOException, ExecutionException, Interrupt
assertHitCount(response, 1);
}
- public void testWithRouting() throws IOException {
+ public void testWithRouting() throws IOException, InterruptedException {
client().prepareIndex("index").setId("1").setRouting("3").setSource("text", "this is a document").get();
client().prepareIndex("index").setId("2").setRouting("1").setSource("text", "this is another document").get();
client().prepareIndex("index").setId("3").setRouting("4").setSource("text", "this is yet another document").get();
refresh("index");
+ indexRandomForConcurrentSearch("index");
Item item = new Item("index", "2").routing("1");
MoreLikeThisQueryBuilder moreLikeThisQueryBuilder = new MoreLikeThisQueryBuilder(
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/msearch/MultiSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/msearch/MultiSearchIT.java
index bc1d2833ecbbf..b35208941d2a2 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/msearch/MultiSearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/msearch/MultiSearchIT.java
@@ -71,12 +71,13 @@ protected Settings featureFlagSettings() {
return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.CONCURRENT_SEGMENT_SEARCH, "true").build();
}
- public void testSimpleMultiSearch() {
+ public void testSimpleMultiSearch() throws InterruptedException {
createIndex("test");
ensureGreen();
client().prepareIndex("test").setId("1").setSource("field", "xxx").get();
client().prepareIndex("test").setId("2").setSource("field", "yyy").get();
refresh();
+ indexRandomForConcurrentSearch("test");
MultiSearchResponse response = client().prepareMultiSearch()
.add(client().prepareSearch("test").setQuery(QueryBuilders.termQuery("field", "xxx")))
.add(client().prepareSearch("test").setQuery(QueryBuilders.termQuery("field", "yyy")))
@@ -94,13 +95,14 @@ public void testSimpleMultiSearch() {
assertFirstHit(response.getResponses()[1].getResponse(), hasId("2"));
}
- public void testSimpleMultiSearchMoreRequests() {
+ public void testSimpleMultiSearchMoreRequests() throws InterruptedException {
createIndex("test");
int numDocs = randomIntBetween(0, 16);
for (int i = 0; i < numDocs; i++) {
client().prepareIndex("test").setId(Integer.toString(i)).setSource("{}", MediaTypeRegistry.JSON).get();
}
refresh();
+ indexRandomForConcurrentSearch("test");
int numSearchRequests = randomIntBetween(1, 64);
MultiSearchRequest request = new MultiSearchRequest();
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedIT.java b/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedIT.java
index 83dec7b27a897..8b375841c2913 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/nested/SimpleNestedIT.java
@@ -126,6 +126,7 @@ public void testSimpleNested() throws Exception {
.get();
waitForRelocation(ClusterHealthStatus.GREEN);
+ indexRandomForConcurrentSearch("test");
GetResponse getResponse = client().prepareGet("test", "1").get();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getSourceAsBytes(), notNullValue());
@@ -293,6 +294,7 @@ public void testMultiNested() throws Exception {
refresh();
// check the numDocs
assertDocumentCount("test", 7);
+ indexRandomForConcurrentSearch("test");
// do some multi nested queries
SearchResponse searchResponse = client().prepareSearch("test")
@@ -485,6 +487,7 @@ public void testExplain() throws Exception {
)
.setRefreshPolicy(IMMEDIATE)
.get();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(nestedQuery("nested1", termQuery("nested1.n_field1", "n_value1"), ScoreMode.Total))
@@ -498,6 +501,10 @@ public void testExplain() throws Exception {
}
public void testSimpleNestedSorting() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setSettings(Settings.builder().put(indexSettings()).put("index.refresh_interval", -1))
.setMapping(
@@ -567,6 +574,7 @@ public void testSimpleNestedSorting() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(QueryBuilders.matchAllQuery())
@@ -596,6 +604,10 @@ public void testSimpleNestedSorting() throws Exception {
}
public void testSimpleNestedSortingWithNestedFilterMissing() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setSettings(Settings.builder().put(indexSettings()).put("index.refresh_interval", -1))
.setMapping(
@@ -675,6 +687,7 @@ public void testSimpleNestedSortingWithNestedFilterMissing() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchRequestBuilder searchRequestBuilder = client().prepareSearch("test")
.setQuery(QueryBuilders.matchAllQuery())
@@ -727,6 +740,10 @@ public void testSimpleNestedSortingWithNestedFilterMissing() throws Exception {
}
public void testNestedSortWithMultiLevelFiltering() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setMapping(
"{\n"
@@ -863,6 +880,7 @@ public void testNestedSortWithMultiLevelFiltering() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
// access id = 1, read, max value, asc, should use grault and quxx
SearchResponse searchResponse = client().prepareSearch()
@@ -968,6 +986,10 @@ public void testNestedSortWithMultiLevelFiltering() throws Exception {
// https://github.com/elastic/elasticsearch/issues/31554
public void testLeakingSortValues() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setSettings(Settings.builder().put("number_of_shards", 1))
.setMapping(
@@ -1035,6 +1057,7 @@ public void testLeakingSortValues() throws Exception {
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(termQuery("_id", 2))
@@ -1056,6 +1079,10 @@ public void testLeakingSortValues() throws Exception {
}
public void testSortNestedWithNestedFilter() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setMapping(
XContentFactory.jsonBuilder()
@@ -1215,6 +1242,7 @@ public void testSortNestedWithNestedFilter() throws Exception {
)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
// Without nested filter
SearchResponse searchResponse = client().prepareSearch()
@@ -1453,6 +1481,10 @@ public void testSortNestedWithNestedFilter() throws Exception {
// Issue #9305
public void testNestedSortingWithNestedFilterAsFilter() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/11187",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
prepareCreate("test").setMapping(
jsonBuilder().startObject()
@@ -1595,6 +1627,7 @@ public void testNestedSortingWithNestedFilterAsFilter() throws Exception {
.get();
assertTrue(indexResponse2.getShardInfo().getSuccessful() > 0);
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.addSort(SortBuilders.fieldSort("users.first").setNestedPath("users").order(SortOrder.ASC))
@@ -1627,6 +1660,7 @@ public void testCheckFixedBitSetCache() throws Exception {
client().prepareIndex("test").setId("1").setSource("field", "value").get();
refresh();
ensureSearchable("test");
+ indexRandomForConcurrentSearch("test");
// No nested mapping yet, there shouldn't be anything in the fixed bit set cache
ClusterStatsResponse clusterStatsResponse = client().admin().cluster().prepareClusterStats().get();
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/pit/DeletePitMultiNodeIT.java b/server/src/internalClusterTest/java/org/opensearch/search/pit/DeletePitMultiNodeIT.java
index 43b7179a335f8..8ae652082f653 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/pit/DeletePitMultiNodeIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/pit/DeletePitMultiNodeIT.java
@@ -93,8 +93,8 @@ public void testDeletePit() throws Exception {
assertTrue(deletePitInfo.isSuccessful());
}
validatePitStats("index", 0, 10);
- /**
- * Checking deleting the same PIT id again results in succeeded
+ /*
+ Checking deleting the same PIT id again results in succeeded
*/
deleteExecute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
deletePITResponse = deleteExecute.get();
@@ -113,8 +113,8 @@ public void testDeletePitWithValidAndDeletedIds() throws Exception {
pitIds.add(pitResponse.getId());
validatePitStats("index", 5, 0);
- /**
- * Delete Pit #1
+ /*
+ Delete Pit #1
*/
DeletePitRequest deletePITRequest = new DeletePitRequest(pitIds);
ActionFuture deleteExecute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
@@ -128,8 +128,8 @@ public void testDeletePitWithValidAndDeletedIds() throws Exception {
pitResponse = execute.get();
pitIds.add(pitResponse.getId());
validatePitStats("index", 5, 5);
- /**
- * Delete PIT with both Ids #1 (which is deleted) and #2 (which is present)
+ /*
+ Delete PIT with both Ids #1 (which is deleted) and #2 (which is present)
*/
deletePITRequest = new DeletePitRequest(pitIds);
deleteExecute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
@@ -165,9 +165,9 @@ public void testDeleteAllPits() throws Exception {
validatePitStats("index1", 5, 0);
DeletePitRequest deletePITRequest = new DeletePitRequest("_all");
- /**
- * When we invoke delete again, returns success after clearing the remaining readers. Asserting reader context
- * not found exceptions don't result in failures ( as deletion in one node is successful )
+ /*
+ When we invoke delete again, returns success after clearing the remaining readers. Asserting reader context
+ not found exceptions don't result in failures ( as deletion in one node is successful )
*/
ActionFuture execute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
DeletePitResponse deletePITResponse = execute.get();
@@ -207,9 +207,9 @@ public Settings onNodeStopped(String nodeName) throws Exception {
});
ensureGreen();
- /**
- * When we invoke delete again, returns success after clearing the remaining readers. Asserting reader context
- * not found exceptions don't result in failures ( as deletion in one node is successful )
+ /*
+ When we invoke delete again, returns success after clearing the remaining readers. Asserting reader context
+ not found exceptions don't result in failures ( as deletion in one node is successful )
*/
ActionFuture execute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
DeletePitResponse deletePITResponse = execute.get();
@@ -242,9 +242,9 @@ public Settings onNodeStopped(String nodeName) throws Exception {
}
});
ensureGreen();
- /**
- * When we invoke delete again, returns success as all readers are cleared. (Delete all on node which is Up and
- * once the node restarts, all active contexts are cleared in the node )
+ /*
+ When we invoke delete again, returns success as all readers are cleared. (Delete all on node which is Up and
+ once the node restarts, all active contexts are cleared in the node )
*/
ActionFuture execute = client().execute(DeletePitAction.INSTANCE, deletePITRequest);
DeletePitResponse deletePITResponse = execute.get();
@@ -278,8 +278,8 @@ public void testDeleteWhileSearch() throws Exception {
}
}
} catch (Exception e) {
- /**
- * assert for exception once delete pit goes through. throw error in case of any exeption before that.
+ /*
+ assert for exception once delete pit goes through. throw error in case of any exeption before that.
*/
if (deleted.get() == true) {
Throwable t = ExceptionsHelper.unwrapCause(e.getCause());
@@ -309,7 +309,11 @@ public void testDeleteWhileSearch() throws Exception {
private void verifySearchContextMissingException(ShardSearchFailure[] failures) {
for (ShardSearchFailure failure : failures) {
Throwable cause = ExceptionsHelper.unwrapCause(failure.getCause());
- assertTrue(failure.toString(), cause instanceof SearchContextMissingException);
+ if (failure.toString().contains("reader_context is already closed can't increment refCount current count")) {
+ // this is fine, expected search error when context is already deleted
+ } else {
+ assertTrue(failure.toString(), cause instanceof SearchContextMissingException);
+ }
}
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/pit/PitMultiNodeIT.java b/server/src/internalClusterTest/java/org/opensearch/search/pit/PitMultiNodeIT.java
index e42f12709c948..a3432bfe7e3e4 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/pit/PitMultiNodeIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/pit/PitMultiNodeIT.java
@@ -100,6 +100,7 @@ public void clearIndex() {
public void testPit() throws Exception {
CreatePitRequest request = new CreatePitRequest(TimeValue.timeValueDays(1), true);
request.setIndices(new String[] { "index" });
+ indexRandomForConcurrentSearch("index");
ActionFuture execute = client().execute(CreatePitAction.INSTANCE, request);
CreatePitResponse pitResponse = execute.get();
SearchResponse searchResponse = client().prepareSearch("index")
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java b/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java
index 425764b1c88d2..6e40c08ed08a1 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java
@@ -52,7 +52,6 @@
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.ParameterizedOpenSearchIntegTestCase;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -99,7 +98,7 @@ public Settings nodeSettings(int nodeOrdinal) {
}
// see #2896
- public void testStopOneNodePreferenceWithRedState() throws IOException {
+ public void testStopOneNodePreferenceWithRedState() throws Exception {
assertAcked(
prepareCreate("test").setSettings(
Settings.builder().put("index.number_of_shards", cluster().numDataNodes() + 2).put("index.number_of_replicas", 0)
@@ -110,6 +109,7 @@ public void testStopOneNodePreferenceWithRedState() throws IOException {
client().prepareIndex("test").setId("" + i).setSource("field1", "value1").get();
}
refresh();
+ indexRandomForConcurrentSearch("test");
internalCluster().stopRandomDataNode();
client().admin().cluster().prepareHealth().setWaitForStatus(ClusterHealthStatus.RED).get();
String[] preferences = new String[] {
@@ -138,7 +138,7 @@ public void testStopOneNodePreferenceWithRedState() throws IOException {
assertThat("_only_local", searchResponse.getFailedShards(), greaterThanOrEqualTo(0));
}
- public void testNoPreferenceRandom() {
+ public void testNoPreferenceRandom() throws Exception {
assertAcked(
prepareCreate("test").setSettings(
// this test needs at least a replica to make sure two consecutive searches go to two different copies of the same data
@@ -149,6 +149,7 @@ public void testNoPreferenceRandom() {
client().prepareIndex("test").setSource("field1", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
final Client client = internalCluster().smartClient();
SearchResponse searchResponse = client.prepareSearch("test").setQuery(matchAllQuery()).get();
@@ -159,12 +160,13 @@ public void testNoPreferenceRandom() {
assertThat(firstNodeId, not(equalTo(secondNodeId)));
}
- public void testSimplePreference() {
+ public void testSimplePreference() throws InterruptedException {
client().admin().indices().prepareCreate("test").setSettings("{\"number_of_replicas\": 1}", MediaTypeRegistry.JSON).get();
ensureGreen();
client().prepareIndex("test").setSource("field1", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).get();
assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L));
@@ -201,7 +203,7 @@ public void testThatSpecifyingNonExistingNodesReturnsUsefulError() {
}
}
- public void testNodesOnlyRandom() {
+ public void testNodesOnlyRandom() throws Exception {
assertAcked(
prepareCreate("test").setSettings(
// this test needs at least a replica to make sure two consecutive searches go to two different copies of the same data
@@ -211,6 +213,7 @@ public void testNodesOnlyRandom() {
ensureGreen();
client().prepareIndex("test").setSource("field1", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
final Client client = internalCluster().smartClient();
// multiple wildchar to cover multi-param usecase
@@ -262,7 +265,7 @@ private void assertSearchOnRandomNodes(SearchRequestBuilder request) {
assertThat(hitNodes.size(), greaterThan(1));
}
- public void testCustomPreferenceUnaffectedByOtherShardMovements() {
+ public void testCustomPreferenceUnaffectedByOtherShardMovements() throws InterruptedException {
/*
* Custom preferences can be used to encourage searches to go to a consistent set of shard copies, meaning that other copies' data
@@ -281,6 +284,7 @@ public void testCustomPreferenceUnaffectedByOtherShardMovements() {
ensureGreen();
client().prepareIndex("test").setSource("field1", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
final String customPreference = randomAlphaOfLength(10);
@@ -300,6 +304,7 @@ public void testCustomPreferenceUnaffectedByOtherShardMovements() {
prepareCreate("test2").setSettings(Settings.builder().put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, replicasInNewIndex))
);
ensureGreen();
+ indexRandomForConcurrentSearch("test2");
assertSearchesSpecificNode("test", customPreference, nodeId);
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/profile/query/QueryProfilerIT.java b/server/src/internalClusterTest/java/org/opensearch/search/profile/query/QueryProfilerIT.java
index 5f794d2abf878..ef73438114079 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/profile/query/QueryProfilerIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/profile/query/QueryProfilerIT.java
@@ -32,6 +32,8 @@
package org.opensearch.search.profile.query;
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+
import org.apache.lucene.tests.util.English;
import org.opensearch.action.index.IndexRequestBuilder;
import org.opensearch.action.search.MultiSearchResponse;
@@ -40,20 +42,23 @@
import org.opensearch.action.search.SearchType;
import org.opensearch.action.search.ShardSearchFailure;
import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.FeatureFlags;
import org.opensearch.index.query.QueryBuilder;
import org.opensearch.index.query.QueryBuilders;
import org.opensearch.search.SearchHit;
import org.opensearch.search.profile.ProfileResult;
import org.opensearch.search.profile.ProfileShardResult;
import org.opensearch.search.sort.SortOrder;
-import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.test.ParameterizedOpenSearchIntegTestCase;
import java.util.Arrays;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING;
import static org.opensearch.search.profile.query.RandomQueryGenerator.randomQueryBuilder;
import static org.hamcrest.Matchers.emptyOrNullString;
import static org.hamcrest.Matchers.equalTo;
@@ -61,8 +66,32 @@
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+
+public class QueryProfilerIT extends ParameterizedOpenSearchIntegTestCase {
+ private final boolean concurrentSearchEnabled;
+ private static final String MAX_PREFIX = "max_";
+ private static final String MIN_PREFIX = "min_";
+ private static final String AVG_PREFIX = "avg_";
+ private static final String TIMING_TYPE_COUNT_SUFFIX = "_count";
+
+ public QueryProfilerIT(Settings settings, boolean concurrentSearchEnabled) {
+ super(settings);
+ this.concurrentSearchEnabled = concurrentSearchEnabled;
+ }
-public class QueryProfilerIT extends OpenSearchIntegTestCase {
+ @ParametersFactory
+ public static Collection parameters() {
+ return Arrays.asList(
+ new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), false).build(), false },
+ new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build(), true }
+ );
+ }
+
+ @Override
+ protected Settings featureFlagSettings() {
+ return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.CONCURRENT_SEGMENT_SEARCH, "true").build();
+ }
/**
* This test simply checks to make sure nothing crashes. Test indexes 100-150 documents,
@@ -229,6 +258,7 @@ public void testSimpleMatch() throws Exception {
assertEquals(result.getLuceneDescription(), "field1:one");
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -271,6 +301,7 @@ public void testBool() throws Exception {
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
assertEquals(result.getProfiledChildren().size(), 2);
+ assertQueryProfileResult(result);
// Check the children
List children = result.getProfiledChildren();
@@ -282,12 +313,14 @@ public void testBool() throws Exception {
assertThat(childProfile.getTime(), greaterThan(0L));
assertNotNull(childProfile.getTimeBreakdown());
assertEquals(childProfile.getProfiledChildren().size(), 0);
+ assertQueryProfileResult(childProfile);
childProfile = children.get(1);
assertEquals(childProfile.getQueryName(), "TermQuery");
assertEquals(childProfile.getLuceneDescription(), "field1:two");
assertThat(childProfile.getTime(), greaterThan(0L));
assertNotNull(childProfile.getTimeBreakdown());
+ assertQueryProfileResult(childProfile);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -330,6 +363,7 @@ public void testEmptyBool() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -375,6 +409,7 @@ public void testCollapsingBool() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -415,6 +450,90 @@ public void testBoosting() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
+ }
+
+ CollectorResult result = searchProfiles.getCollectorResult();
+ assertThat(result.getName(), is(not(emptyOrNullString())));
+ assertThat(result.getTime(), greaterThan(0L));
+ }
+ }
+ }
+
+ public void testSearchLeafForItsLeavesAndRewriteQuery() throws Exception {
+ createIndex("test");
+ ensureGreen();
+
+ int numDocs = 122;
+ IndexRequestBuilder[] docs = new IndexRequestBuilder[numDocs];
+ for (int i = 0; i < numDocs; i++) {
+ docs[i] = client().prepareIndex("test").setId(String.valueOf(i)).setSource("field1", English.intToEnglish(i), "field2", i);
+ }
+
+ List terms = Arrays.asList("zero", "zero", "one");
+
+ indexRandom(true, docs);
+
+ refresh();
+
+ QueryBuilder q = QueryBuilders.boostingQuery(
+ QueryBuilders.idsQuery().addIds(String.valueOf(randomInt()), String.valueOf(randomInt())),
+ QueryBuilders.termsQuery("field1", terms)
+ ).boost(randomFloat()).negativeBoost(randomFloat());
+ logger.info("Query: {}", q);
+
+ SearchResponse resp = client().prepareSearch()
+ .setQuery(q)
+ .setTrackTotalHits(true)
+ .setProfile(true)
+ .setSearchType(SearchType.QUERY_THEN_FETCH)
+ .get();
+
+ assertNotNull("Profile response element should not be null", resp.getProfileResults());
+ assertThat("Profile response should not be an empty array", resp.getProfileResults().size(), not(0));
+
+ for (Map.Entry shardResult : resp.getProfileResults().entrySet()) {
+ assertThat(shardResult.getValue().getNetworkTime().getInboundNetworkTime(), greaterThanOrEqualTo(0L));
+ assertThat(shardResult.getValue().getNetworkTime().getOutboundNetworkTime(), greaterThanOrEqualTo(0L));
+ for (QueryProfileShardResult searchProfiles : shardResult.getValue().getQueryProfileResults()) {
+ List results = searchProfiles.getQueryResults();
+ for (ProfileResult result : results) {
+ assertNotNull(result.getQueryName());
+ assertNotNull(result.getLuceneDescription());
+ assertThat(result.getTime(), greaterThan(0L));
+ Map breakdown = result.getTimeBreakdown();
+ Long maxSliceTime = result.getMaxSliceTime();
+ Long minSliceTime = result.getMinSliceTime();
+ Long avgSliceTime = result.getAvgSliceTime();
+ if (concurrentSearchEnabled && results.get(0).equals(result)) {
+ assertNotNull(maxSliceTime);
+ assertNotNull(minSliceTime);
+ assertNotNull(avgSliceTime);
+ assertThat(breakdown.size(), equalTo(66));
+ for (QueryTimingType queryTimingType : QueryTimingType.values()) {
+ if (queryTimingType != QueryTimingType.CREATE_WEIGHT) {
+ String maxTimingType = MAX_PREFIX + queryTimingType;
+ String minTimingType = MIN_PREFIX + queryTimingType;
+ String avgTimingType = AVG_PREFIX + queryTimingType;
+ assertNotNull(breakdown.get(maxTimingType));
+ assertNotNull(breakdown.get(minTimingType));
+ assertNotNull(breakdown.get(avgTimingType));
+ assertNotNull(breakdown.get(maxTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ assertNotNull(breakdown.get(minTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ assertNotNull(breakdown.get(avgTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ }
+ }
+ } else if (concurrentSearchEnabled) {
+ assertThat(maxSliceTime, equalTo(0L));
+ assertThat(minSliceTime, equalTo(0L));
+ assertThat(avgSliceTime, equalTo(0L));
+ assertThat(breakdown.size(), equalTo(27));
+ } else {
+ assertThat(maxSliceTime, is(nullValue()));
+ assertThat(minSliceTime, is(nullValue()));
+ assertThat(avgSliceTime, is(nullValue()));
+ assertThat(breakdown.size(), equalTo(27));
+ }
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -455,6 +574,7 @@ public void testDisMaxRange() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -494,6 +614,7 @@ public void testRange() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -547,6 +668,7 @@ public void testPhrase() throws Exception {
assertNotNull(result.getLuceneDescription());
assertThat(result.getTime(), greaterThan(0L));
assertNotNull(result.getTimeBreakdown());
+ assertQueryProfileResult(result);
}
CollectorResult result = searchProfiles.getCollectorResult();
@@ -579,4 +701,35 @@ public void testNoProfile() throws Exception {
assertThat("Profile response element should be an empty map", resp.getProfileResults().size(), equalTo(0));
}
+ private void assertQueryProfileResult(ProfileResult result) {
+ Map breakdown = result.getTimeBreakdown();
+ Long maxSliceTime = result.getMaxSliceTime();
+ Long minSliceTime = result.getMinSliceTime();
+ Long avgSliceTime = result.getAvgSliceTime();
+ if (concurrentSearchEnabled) {
+ assertNotNull(maxSliceTime);
+ assertNotNull(minSliceTime);
+ assertNotNull(avgSliceTime);
+ assertThat(breakdown.size(), equalTo(66));
+ for (QueryTimingType queryTimingType : QueryTimingType.values()) {
+ if (queryTimingType != QueryTimingType.CREATE_WEIGHT) {
+ String maxTimingType = MAX_PREFIX + queryTimingType;
+ String minTimingType = MIN_PREFIX + queryTimingType;
+ String avgTimingType = AVG_PREFIX + queryTimingType;
+ assertNotNull(breakdown.get(maxTimingType));
+ assertNotNull(breakdown.get(minTimingType));
+ assertNotNull(breakdown.get(avgTimingType));
+ assertNotNull(breakdown.get(maxTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ assertNotNull(breakdown.get(minTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ assertNotNull(breakdown.get(avgTimingType + TIMING_TYPE_COUNT_SUFFIX));
+ }
+ }
+ } else {
+ assertThat(maxSliceTime, is(nullValue()));
+ assertThat(minSliceTime, is(nullValue()));
+ assertThat(avgSliceTime, is(nullValue()));
+ assertThat(breakdown.size(), equalTo(27));
+ }
+ }
+
}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java
index 099eb934f4f4d..1ca5859f23bca 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java
@@ -186,6 +186,7 @@ public void testDocWithAllTypes() throws Exception {
String docBody = copyToStringFromClasspath("/org/opensearch/search/query/all-example-document.json");
reqs.add(client().prepareIndex("test").setId("1").setSource(docBody, MediaTypeRegistry.JSON));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(queryStringQuery("foo")).get();
assertHits(resp.getHits(), "1");
@@ -225,6 +226,7 @@ public void testKeywordWithWhitespace() throws Exception {
reqs.add(client().prepareIndex("test").setId("2").setSource("f1", "bar"));
reqs.add(client().prepareIndex("test").setId("3").setSource("f1", "foo bar"));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(queryStringQuery("foo")).get();
assertHits(resp.getHits(), "3");
@@ -245,6 +247,7 @@ public void testRegexCaseInsensitivity() throws Exception {
indexRequests.add(client().prepareIndex("messages").setId("1").setSource("message", "message: this is a TLS handshake"));
indexRequests.add(client().prepareIndex("messages").setId("2").setSource("message", "message: this is a tcp handshake"));
indexRandom(true, false, indexRequests);
+ indexRandomForConcurrentSearch("messages");
SearchResponse response = client().prepareSearch("messages").setQuery(queryStringQuery("/TLS/").defaultField("message")).get();
assertNoFailures(response);
@@ -282,6 +285,7 @@ public void testAllFields() throws Exception {
List reqs = new ArrayList<>();
reqs.add(client().prepareIndex("test_1").setId("1").setSource("f1", "foo", "f2", "eggplant"));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test_1");
SearchResponse resp = client().prepareSearch("test_1")
.setQuery(queryStringQuery("foo eggplant").defaultOperator(Operator.AND))
@@ -374,6 +378,7 @@ public void testLimitOnExpandedFields() throws Exception {
client().prepareIndex("testindex").setId("1").setSource("field_A0", "foo bar baz").get();
refresh();
+ indexRandomForConcurrentSearch("testindex");
// single field shouldn't trigger the limit
doAssertOneHitForQueryString("field_A0:foo");
@@ -465,6 +470,7 @@ public void testFieldAliasOnDisallowedFieldType() throws Exception {
List indexRequests = new ArrayList<>();
indexRequests.add(client().prepareIndex("test").setId("1").setSource("f3", "text", "f2", "one"));
indexRandom(true, false, indexRequests);
+ indexRandomForConcurrentSearch("test");
// The wildcard field matches aliases for both a text and geo_point field.
// By default, the geo_point field should be ignored when building the query.
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/ScriptScoreQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/ScriptScoreQueryIT.java
index 7ba582811bbc2..55029712a061c 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/query/ScriptScoreQueryIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/query/ScriptScoreQueryIT.java
@@ -109,13 +109,14 @@ protected Map, Object>> pluginScripts() {
// 1) only matched docs retrieved
// 2) score is calculated based on a script with params
// 3) min score applied
- public void testScriptScore() {
+ public void testScriptScore() throws Exception {
assertAcked(prepareCreate("test-index").setMapping("field1", "type=text", "field2", "type=double"));
int docCount = 10;
for (int i = 1; i <= docCount; i++) {
client().prepareIndex("test-index").setId("" + i).setSource("field1", "text" + (i % 2), "field2", i).get();
}
refresh();
+ indexRandomForConcurrentSearch("test-index");
Map params = new HashMap<>();
params.put("param1", 0.1);
@@ -135,13 +136,14 @@ public void testScriptScore() {
assertOrderedSearchHits(resp, "10", "8", "6");
}
- public void testScriptScoreBoolQuery() {
+ public void testScriptScoreBoolQuery() throws Exception {
assertAcked(prepareCreate("test-index").setMapping("field1", "type=text", "field2", "type=double"));
int docCount = 10;
for (int i = 1; i <= docCount; i++) {
client().prepareIndex("test-index").setId("" + i).setSource("field1", "text" + i, "field2", i).get();
}
refresh();
+ indexRandomForConcurrentSearch("test-index");
Map params = new HashMap<>();
params.put("param1", 0.1);
@@ -155,7 +157,7 @@ public void testScriptScoreBoolQuery() {
}
// test that when the internal query is rewritten script_score works well
- public void testRewrittenQuery() {
+ public void testRewrittenQuery() throws Exception {
assertAcked(
prepareCreate("test-index2").setSettings(Settings.builder().put("index.number_of_shards", 1))
.setMapping("field1", "type=date", "field2", "type=double")
@@ -164,6 +166,7 @@ public void testRewrittenQuery() {
client().prepareIndex("test-index2").setId("2").setSource("field1", "2019-10-01", "field2", 2).get();
client().prepareIndex("test-index2").setId("3").setSource("field1", "2019-11-01", "field2", 3).get();
refresh();
+ indexRandomForConcurrentSearch("test-index2");
RangeQueryBuilder rangeQB = new RangeQueryBuilder("field1").from("2019-01-01"); // the query should be rewritten to from:null
Map params = new HashMap<>();
@@ -174,7 +177,7 @@ public void testRewrittenQuery() {
assertOrderedSearchHits(resp, "3", "2", "1");
}
- public void testDisallowExpensiveQueries() {
+ public void testDisallowExpensiveQueries() throws Exception {
try {
assertAcked(prepareCreate("test-index").setMapping("field1", "type=text", "field2", "type=double"));
int docCount = 10;
@@ -182,6 +185,7 @@ public void testDisallowExpensiveQueries() {
client().prepareIndex("test-index").setId("" + i).setSource("field1", "text" + (i % 2), "field2", i).get();
}
refresh();
+ indexRandomForConcurrentSearch("test-index");
Map params = new HashMap<>();
params.put("param1", 0.1);
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java
index 53bded1fc493c..d2bca41760ff6 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java
@@ -465,6 +465,7 @@ public void testQueryStringAnalyzedWildcard() throws Exception {
client().prepareIndex("test").setId("1").setSource("field1", "value_1", "field2", "value_2").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(queryStringQuery("value*")).get();
assertHitCount(searchResponse, 1L);
@@ -482,11 +483,12 @@ public void testQueryStringAnalyzedWildcard() throws Exception {
assertHitCount(searchResponse, 1L);
}
- public void testLowercaseExpandedTerms() {
+ public void testLowercaseExpandedTerms() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field1", "value_1", "field2", "value_2").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(queryStringQuery("VALUE_3~1")).get();
assertHitCount(searchResponse, 1L);
@@ -499,7 +501,7 @@ public void testLowercaseExpandedTerms() {
}
// Issue #3540
- public void testDateRangeInQueryString() {
+ public void testDateRangeInQueryString() throws InterruptedException {
// the mapping needs to be provided upfront otherwise we are not sure how many failures we get back
// as with dynamic mappings some shards might be lacking behind and parse a different query
assertAcked(prepareCreate("test").setMapping("past", "type=date", "future", "type=date"));
@@ -510,6 +512,7 @@ public void testDateRangeInQueryString() {
client().prepareIndex("test").setId("1").setSource("past", aMonthAgo, "future", aMonthFromNow).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(queryStringQuery("past:[now-2M/d TO now/d]")).get();
assertHitCount(searchResponse, 1L);
@@ -525,7 +528,7 @@ public void testDateRangeInQueryString() {
}
// Issue #7880
- public void testDateRangeInQueryStringWithTimeZone_7880() {
+ public void testDateRangeInQueryStringWithTimeZone_7880() throws InterruptedException {
// the mapping needs to be provided upfront otherwise we are not sure how many failures we get back
// as with dynamic mappings some shards might be lacking behind and parse a different query
assertAcked(prepareCreate("test").setMapping("past", "type=date"));
@@ -536,6 +539,7 @@ public void testDateRangeInQueryStringWithTimeZone_7880() {
client().prepareIndex("test").setId("1").setSource("past", now).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(queryStringQuery("past:[now-1m/m TO now+1m/m]").timeZone(timeZone.getId()))
.get();
@@ -543,7 +547,7 @@ public void testDateRangeInQueryStringWithTimeZone_7880() {
}
// Issue #10477
- public void testDateRangeInQueryStringWithTimeZone_10477() {
+ public void testDateRangeInQueryStringWithTimeZone_10477() throws InterruptedException {
// the mapping needs to be provided upfront otherwise we are not sure how many failures we get back
// as with dynamic mappings some shards might be lacking behind and parse a different query
assertAcked(prepareCreate("test").setMapping("past", "type=date"));
@@ -552,6 +556,7 @@ public void testDateRangeInQueryStringWithTimeZone_10477() {
client().prepareIndex("test").setId("2").setSource("past", "2015-04-06T00:00:00+0000").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// Timezone set with dates
SearchResponse searchResponse = client().prepareSearch()
.setQuery(queryStringQuery("past:[2015-04-06T00:00:00+0200 TO 2015-04-06T23:00:00+0200]"))
@@ -725,6 +730,7 @@ public void testPassQueryOrFilterAsJSONString() throws Exception {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field1", "value1_1", "field2", "value2_1").setRefreshPolicy(IMMEDIATE).get();
+ indexRandomForConcurrentSearch("test");
WrapperQueryBuilder wrapper = new WrapperQueryBuilder("{ \"term\" : { \"field1\" : \"value1_1\" } }");
assertHitCount(client().prepareSearch().setQuery(wrapper).get(), 1L);
@@ -741,6 +747,7 @@ public void testFiltersWithCustomCacheKey() throws Exception {
client().prepareIndex("test").setId("1").setSource("field1", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test").setQuery(constantScoreQuery(termsQuery("field1", "value1"))).get();
assertHitCount(searchResponse, 1L);
@@ -782,6 +789,7 @@ public void testMatchQueryFuzzy() throws Exception {
client().prepareIndex("test").setId("1").setSource("text", "Unit"),
client().prepareIndex("test").setId("2").setSource("text", "Unity")
);
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(matchQuery("text", "uniy").fuzziness(Fuzziness.ZERO)).get();
assertHitCount(searchResponse, 0L);
@@ -913,11 +921,12 @@ public void testMultiMatchQueryZeroTermsQuery() {
assertHitCount(searchResponse, 2L);
}
- public void testMultiMatchQueryMinShouldMatch() {
+ public void testMultiMatchQueryMinShouldMatch() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field1", new String[] { "value1", "value2", "value3" }).get();
client().prepareIndex("test").setId("2").setSource("field2", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
MultiMatchQueryBuilder multiMatchQuery = multiMatchQuery("value1 value2 foo", "field1", "field2");
@@ -959,12 +968,13 @@ public void testMultiMatchQueryMinShouldMatch() {
assertHitCount(searchResponse, 0L);
}
- public void testBoolQueryMinShouldMatchBiggerThanNumberOfShouldClauses() throws IOException {
+ public void testBoolQueryMinShouldMatchBiggerThanNumberOfShouldClauses() throws IOException, InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field1", new String[] { "value1", "value2", "value3" }).get();
client().prepareIndex("test").setId("2").setSource("field2", "value1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
BoolQueryBuilder boolQuery = boolQuery().must(termQuery("field1", "value1"))
.should(boolQuery().should(termQuery("field1", "value1")).should(termQuery("field1", "value2")).minimumShouldMatch(3));
SearchResponse searchResponse = client().prepareSearch().setQuery(boolQuery).get();
@@ -991,12 +1001,13 @@ public void testBoolQueryMinShouldMatchBiggerThanNumberOfShouldClauses() throws
assertHitCount(searchResponse, 0L);
}
- public void testFuzzyQueryString() {
+ public void testFuzzyQueryString() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("str", "foobar", "date", "2012-02-01", "num", 12).get();
client().prepareIndex("test").setId("2").setSource("str", "fred", "date", "2012-02-05", "num", 20).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(queryStringQuery("str:foobaz~1")).get();
assertNoFailures(searchResponse);
assertHitCount(searchResponse, 1L);
@@ -1015,6 +1026,7 @@ public void testQuotedQueryStringWithBoost() throws InterruptedException {
client().prepareIndex("test").setId("2").setSource("important", "nothing important", "less_important", "phrase match")
);
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch()
.setQuery(queryStringQuery("\"phrase match\"").field("important", boost).field("less_important"))
.get();
@@ -1027,11 +1039,12 @@ public void testQuotedQueryStringWithBoost() throws InterruptedException {
);
}
- public void testSpecialRangeSyntaxInQueryString() {
+ public void testSpecialRangeSyntaxInQueryString() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("str", "foobar", "date", "2012-02-01", "num", 12).get();
client().prepareIndex("test").setId("2").setSource("str", "fred", "date", "2012-02-05", "num", 20).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(queryStringQuery("num:>19")).get();
assertHitCount(searchResponse, 1L);
@@ -1137,6 +1150,7 @@ public void testTermsQuery() throws Exception {
public void testTermsLookupFilter() throws Exception {
assertAcked(prepareCreate("lookup").setMapping("terms", "type=text", "other", "type=text"));
+ indexRandomForConcurrentSearch("lookup");
assertAcked(
prepareCreate("lookup2").setMapping(
jsonBuilder().startObject()
@@ -1152,8 +1166,11 @@ public void testTermsLookupFilter() throws Exception {
.endObject()
)
);
+ indexRandomForConcurrentSearch("lookup2");
assertAcked(prepareCreate("lookup3").setMapping("_source", "enabled=false", "terms", "type=text"));
+ indexRandomForConcurrentSearch("lookup3");
assertAcked(prepareCreate("test").setMapping("term", "type=text"));
+ indexRandomForConcurrentSearch("test");
indexRandom(
true,
@@ -1279,6 +1296,7 @@ public void testBasicQueryById() throws Exception {
client().prepareIndex("test").setId("3").setSource("field1", "value3").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(idsQuery().addIds("1", "2")).get();
assertHitCount(searchResponse, 2L);
assertThat(searchResponse.getHits().getHits().length, equalTo(2));
@@ -1333,6 +1351,7 @@ public void testNumericTermsAndRanges() throws Exception {
.setSource("num_byte", 17, "num_short", 17, "num_integer", 17, "num_long", 17, "num_float", 17, "num_double", 17)
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse;
logger.info("--> term query on 1");
@@ -1439,6 +1458,7 @@ public void testNumericRangeFilter_2826() throws Exception {
client().prepareIndex("test").setId("3").setSource("field1", "test2", "num_long", 3).get();
client().prepareIndex("test").setId("4").setSource("field1", "test2", "num_long", 4).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setPostFilter(boolQuery().should(rangeQuery("num_long").from(1).to(2)).should(rangeQuery("num_long").from(3).to(4)))
@@ -1535,7 +1555,7 @@ public void testSimpleSpan() throws IOException, ExecutionException, Interrupted
assertHitCount(searchResponse, 3L);
}
- public void testSpanMultiTermQuery() throws IOException {
+ public void testSpanMultiTermQuery() throws IOException, InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("description", "foo other anything bar", "count", 1).get();
@@ -1543,6 +1563,7 @@ public void testSpanMultiTermQuery() throws IOException {
client().prepareIndex("test").setId("3").setSource("description", "foo other", "count", 3).get();
client().prepareIndex("test").setId("4").setSource("description", "fop", "count", 4).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setQuery(spanOrQuery(spanMultiTermQueryBuilder(fuzzyQuery("description", "fop"))))
@@ -1574,6 +1595,7 @@ public void testSpanNot() throws IOException, ExecutionException, InterruptedExc
client().prepareIndex("test").setId("1").setSource("description", "the quick brown fox jumped over the lazy dog").get();
client().prepareIndex("test").setId("2").setSource("description", "the quick black fox leaped over the sleeping dog").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(
@@ -1612,7 +1634,7 @@ public void testSpanNot() throws IOException, ExecutionException, InterruptedExc
assertHitCount(searchResponse, 1L);
}
- public void testSimpleDFSQuery() throws IOException {
+ public void testSimpleDFSQuery() throws IOException, InterruptedException {
assertAcked(
prepareCreate("test").setMapping(
jsonBuilder().startObject()
@@ -1657,6 +1679,7 @@ public void testSimpleDFSQuery() throws IOException {
.setSource("online", true, "ts", System.currentTimeMillis() - 123123, "type", "bs")
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
@@ -1679,8 +1702,9 @@ public void testSimpleDFSQuery() throws IOException {
assertNoFailures(response);
}
- public void testMultiFieldQueryString() {
+ public void testMultiFieldQueryString() throws InterruptedException {
client().prepareIndex("test").setId("1").setSource("field1", "value1", "field2", "value2").setRefreshPolicy(IMMEDIATE).get();
+ indexRandomForConcurrentSearch("test");
logger.info("regular");
assertHitCount(client().prepareSearch("test").setQuery(queryStringQuery("value1").field("field1").field("field2")).get(), 1);
@@ -1700,11 +1724,12 @@ public void testMultiFieldQueryString() {
}
// see #3797
- public void testMultiMatchLenientIssue3797() {
+ public void testMultiMatchLenientIssue3797() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field1", 123, "field2", "value2").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(multiMatchQuery("value2", "field2").field("field1", 2).lenient(true))
@@ -1728,6 +1753,7 @@ public void testMinScore() throws ExecutionException, InterruptedException {
client().prepareIndex("test").setId("3").setSource("score", 2.0).get();
client().prepareIndex("test").setId("4").setSource("score", 0.5).get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(functionScoreQuery(ScoreFunctionBuilders.fieldValueFactorFunction("score").missing(1.0)).setMinScore(1.5f))
@@ -1737,12 +1763,13 @@ public void testMinScore() throws ExecutionException, InterruptedException {
assertSecondHit(searchResponse, hasId("1"));
}
- public void testQueryStringWithSlopAndFields() {
+ public void testQueryStringWithSlopAndFields() throws InterruptedException {
assertAcked(prepareCreate("test"));
client().prepareIndex("test").setId("1").setSource("desc", "one two three", "type", "customer").get();
client().prepareIndex("test").setId("2").setSource("desc", "one two three", "type", "product").get();
refresh();
+ indexRandomForConcurrentSearch("test");
{
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(QueryBuilders.queryStringQuery("\"one two\"").defaultField("desc"))
@@ -1809,6 +1836,7 @@ public void testRangeQueryWithTimeZone() throws Exception {
.setId("4")
.setSource("date", Instant.now().atZone(ZoneOffset.ofHours(1)).toInstant().toEpochMilli(), "num", 4)
);
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(QueryBuilders.rangeQuery("date").from("2014-01-01T00:00:00").to("2014-01-01T00:59:00"))
@@ -1948,6 +1976,7 @@ public void testMatchPhrasePrefixQuery() throws ExecutionException, InterruptedE
public void testQueryStringParserCache() throws Exception {
createIndex("test");
indexRandom(true, false, client().prepareIndex("test").setId("1").setSource("nameTokens", "xyz"));
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test")
.setSearchType(SearchType.DFS_QUERY_THEN_FETCH)
@@ -1978,6 +2007,7 @@ public void testRangeQueryRangeFields_24744() throws Exception {
.setSource(jsonBuilder().startObject().startObject("int_range").field("gte", 10).field("lte", 20).endObject().endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
RangeQueryBuilder range = new RangeQueryBuilder("int_range").relation("intersects").from(Integer.MIN_VALUE).to(Integer.MAX_VALUE);
SearchResponse searchResponse = client().prepareSearch("test").setQuery(range).get();
@@ -2013,6 +2043,7 @@ public void testNestedQueryWithFieldAlias() throws Exception {
index("index", "_doc", "1", source);
refresh();
+ indexRandomForConcurrentSearch("index");
QueryBuilder nestedQuery = QueryBuilders.nestedQuery(
"section",
@@ -2041,6 +2072,7 @@ public void testFieldAliasesForMetaFields() throws Exception {
IndexRequestBuilder indexRequest = client().prepareIndex("test").setId("1").setRouting("custom").setSource("field", "value");
indexRandom(true, false, indexRequest);
+ indexRandomForConcurrentSearch("test");
client().admin()
.cluster()
.prepareUpdateSettings()
@@ -2073,7 +2105,7 @@ public void testFieldAliasesForMetaFields() throws Exception {
/**
* Test that wildcard queries on keyword fields get normalized
*/
- public void testWildcardQueryNormalizationOnKeywordField() {
+ public void testWildcardQueryNormalizationOnKeywordField() throws InterruptedException {
assertAcked(
prepareCreate("test").setSettings(
Settings.builder()
@@ -2084,6 +2116,7 @@ public void testWildcardQueryNormalizationOnKeywordField() {
);
client().prepareIndex("test").setId("1").setSource("field1", "Bbb Aaa").get();
refresh();
+ indexRandomForConcurrentSearch("test");
{
WildcardQueryBuilder wildCardQuery = wildcardQuery("field1", "Bb*");
@@ -2099,7 +2132,7 @@ public void testWildcardQueryNormalizationOnKeywordField() {
/**
* Test that wildcard queries on text fields get normalized
*/
- public void testWildcardQueryNormalizationOnTextField() {
+ public void testWildcardQueryNormalizationOnTextField() throws InterruptedException {
assertAcked(
prepareCreate("test").setSettings(
Settings.builder()
@@ -2111,6 +2144,7 @@ public void testWildcardQueryNormalizationOnTextField() {
);
client().prepareIndex("test").setId("1").setSource("field1", "Bbb Aaa").get();
refresh();
+ indexRandomForConcurrentSearch("test");
{
// test default case insensitivity: false
@@ -2130,10 +2164,11 @@ public void testWildcardQueryNormalizationOnTextField() {
}
/** tests wildcard case sensitivity */
- public void testWildcardCaseSensitivity() {
+ public void testWildcardCaseSensitivity() throws InterruptedException {
assertAcked(prepareCreate("test").setMapping("field", "type=text"));
client().prepareIndex("test").setId("1").setSource("field", "lowercase text").get();
refresh();
+ indexRandomForConcurrentSearch("test");
// test case sensitive
SearchResponse response = client().prepareSearch("test").setQuery(wildcardQuery("field", "Text").caseInsensitive(false)).get();
@@ -2151,7 +2186,7 @@ public void testWildcardCaseSensitivity() {
* Reserved characters should be excluded when the normalization is applied for keyword fields.
* See https://github.com/elastic/elasticsearch/issues/46300 for details.
*/
- public void testWildcardQueryNormalizationKeywordSpecialCharacters() {
+ public void testWildcardQueryNormalizationKeywordSpecialCharacters() throws InterruptedException {
assertAcked(
prepareCreate("test").setSettings(
Settings.builder()
@@ -2163,6 +2198,7 @@ public void testWildcardQueryNormalizationKeywordSpecialCharacters() {
);
client().prepareIndex("test").setId("1").setSource("field", "label-1").get();
refresh();
+ indexRandomForConcurrentSearch("test");
WildcardQueryBuilder wildCardQuery = wildcardQuery("field", "la*");
SearchResponse searchResponse = client().prepareSearch().setQuery(wildCardQuery).get();
@@ -2213,11 +2249,12 @@ public Map> getTokenizers() {
* set for fuzzy queries with "constant_score" rewrite nested inside a `span_multi` query and would cause NPEs due to an unset
* {@link AttributeSource}.
*/
- public void testIssueFuzzyInsideSpanMulti() {
+ public void testIssueFuzzyInsideSpanMulti() throws InterruptedException {
createIndex("test");
client().prepareIndex("test").setId("1").setSource("field", "foobarbaz").get();
ensureGreen();
refresh();
+ indexRandomForConcurrentSearch("test");
BoolQueryBuilder query = boolQuery().filter(spanMultiTermQueryBuilder(fuzzyQuery("field", "foobarbiz").rewrite("constant_score")));
SearchResponse response = client().prepareSearch("test").setQuery(query).get();
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java
index 384d2b7423e66..d8902238005da 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java
@@ -150,6 +150,7 @@ public void testSimpleQueryString() throws ExecutionException, InterruptedExcept
client().prepareIndex("test").setId("5").setSource("body", "quux baz spaghetti"),
client().prepareIndex("test").setId("6").setSource("otherbody", "spaghetti")
);
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar")).get();
assertHitCount(searchResponse, 3L);
@@ -199,6 +200,7 @@ public void testSimpleQueryStringMinimumShouldMatch() throws Exception {
client().prepareIndex("test").setId("3").setSource("body", "foo bar"),
client().prepareIndex("test").setId("4").setSource("body", "foo baz bar")
);
+ indexRandomForConcurrentSearch("test");
logger.info("--> query 1");
SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar").minimumShouldMatch("2")).get();
@@ -235,6 +237,7 @@ public void testSimpleQueryStringMinimumShouldMatch() throws Exception {
client().prepareIndex("test").setId("7").setSource("body2", "foo bar", "other", "foo"),
client().prepareIndex("test").setId("8").setSource("body2", "foo baz bar", "other", "foo")
);
+ indexRandomForConcurrentSearch("test");
logger.info("--> query 5");
searchResponse = client().prepareSearch()
@@ -256,7 +259,7 @@ public void testSimpleQueryStringMinimumShouldMatch() throws Exception {
assertSearchHits(searchResponse, "6", "7", "8");
}
- public void testNestedFieldSimpleQueryString() throws IOException {
+ public void testNestedFieldSimpleQueryString() throws Exception {
assertAcked(
prepareCreate("test").setMapping(
jsonBuilder().startObject()
@@ -275,6 +278,7 @@ public void testNestedFieldSimpleQueryString() throws IOException {
);
client().prepareIndex("test").setId("1").setSource("body", "foo bar baz").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar baz").field("body")).get();
assertHitCount(searchResponse, 1L);
@@ -359,6 +363,8 @@ public void testSimpleQueryStringLenient() throws ExecutionException, Interrupte
client().prepareIndex("test2").setId("10").setSource("field", 5)
);
refresh();
+ indexRandomForConcurrentSearch("test1");
+ indexRandomForConcurrentSearch("test2");
SearchResponse searchResponse = client().prepareSearch()
.setAllowPartialSearchResults(true)
@@ -419,6 +425,7 @@ public void testSimpleQueryStringUsesFieldAnalyzer() throws Exception {
client().prepareIndex("test").setId("2").setSource("foo", 234, "bar", "bcd").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("123").field("foo").field("bar")).get();
assertHitCount(searchResponse, 1L);
@@ -430,6 +437,7 @@ public void testSimpleQueryStringOnIndexMetaField() throws Exception {
client().prepareIndex("test").setId("2").setSource("foo", 234, "bar", "bcd").get();
refresh();
+ indexRandomForConcurrentSearch("test");
SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("test").field("_index")).get();
assertHitCount(searchResponse, 2L);
@@ -469,6 +477,7 @@ public void testBasicAllQuery() throws Exception {
reqs.add(client().prepareIndex("test").setId("2").setSource("f2", "Bar"));
reqs.add(client().prepareIndex("test").setId("3").setSource("f3", "foo bar baz"));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(simpleQueryStringQuery("foo")).get();
assertHitCount(resp, 2L);
@@ -492,6 +501,7 @@ public void testWithDate() throws Exception {
reqs.add(client().prepareIndex("test").setId("1").setSource("f1", "foo", "f_date", "2015/09/02"));
reqs.add(client().prepareIndex("test").setId("2").setSource("f1", "bar", "f_date", "2015/09/01"));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(simpleQueryStringQuery("foo bar")).get();
assertHits(resp.getHits(), "1", "2");
@@ -523,6 +533,7 @@ public void testWithLotsOfTypes() throws Exception {
client().prepareIndex("test").setId("2").setSource("f1", "bar", "f_date", "2015/09/01", "f_float", "1.8", "f_ip", "127.0.0.2")
);
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(simpleQueryStringQuery("foo bar")).get();
assertHits(resp.getHits(), "1", "2");
@@ -550,6 +561,7 @@ public void testDocWithAllTypes() throws Exception {
String docBody = copyToStringFromClasspath("/org/opensearch/search/query/all-example-document.json");
reqs.add(client().prepareIndex("test").setId("1").setSource(docBody, MediaTypeRegistry.JSON));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(simpleQueryStringQuery("foo")).get();
assertHits(resp.getHits(), "1");
@@ -596,6 +608,7 @@ public void testKeywordWithWhitespace() throws Exception {
reqs.add(client().prepareIndex("test").setId("2").setSource("f1", "bar"));
reqs.add(client().prepareIndex("test").setId("3").setSource("f1", "foo bar"));
indexRandom(true, false, reqs);
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = client().prepareSearch("test").setQuery(simpleQueryStringQuery("foo")).get();
assertHits(resp.getHits(), "3");
@@ -663,6 +676,7 @@ public void testFieldAlias() throws Exception {
indexRequests.add(client().prepareIndex("test").setId("2").setSource("f3", "value", "f2", "two"));
indexRequests.add(client().prepareIndex("test").setId("3").setSource("f3", "another value", "f2", "three"));
indexRandom(true, false, indexRequests);
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test").setQuery(simpleQueryStringQuery("value").field("f3_alias")).get();
@@ -681,6 +695,7 @@ public void testFieldAliasWithWildcardField() throws Exception {
indexRequests.add(client().prepareIndex("test").setId("2").setSource("f3", "value", "f2", "two"));
indexRequests.add(client().prepareIndex("test").setId("3").setSource("f3", "another value", "f2", "three"));
indexRandom(true, false, indexRequests);
+ indexRandomForConcurrentSearch("test");
SearchResponse response = client().prepareSearch("test").setQuery(simpleQueryStringQuery("value").field("f3_*")).get();
@@ -697,6 +712,7 @@ public void testFieldAliasOnDisallowedFieldType() throws Exception {
List indexRequests = new ArrayList<>();
indexRequests.add(client().prepareIndex("test").setId("1").setSource("f3", "text", "f2", "one"));
indexRandom(true, false, indexRequests);
+ indexRandomForConcurrentSearch("test");
// The wildcard field matches aliases for both a text and boolean field.
// By default, the boolean field should be ignored when building the query.
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/scriptfilter/ScriptQuerySearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/scriptfilter/ScriptQuerySearchIT.java
index 34967528f2c4f..ae00904f237a5 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/scriptfilter/ScriptQuerySearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/scriptfilter/ScriptQuerySearchIT.java
@@ -161,6 +161,7 @@ public void testCustomScriptBinaryField() throws Exception {
.get();
flush();
refresh();
+ indexRandomForConcurrentSearch("my-index");
SearchResponse response = client().prepareSearch()
.setQuery(
@@ -213,6 +214,7 @@ public void testCustomScriptBoost() throws Exception {
.setSource(jsonBuilder().startObject().field("test", "value beck").field("num1", 3.0f).endObject())
.get();
refresh();
+ indexRandomForConcurrentSearch("test");
logger.info("running doc['num1'].value > 1");
SearchResponse response = client().prepareSearch()
@@ -259,7 +261,7 @@ public void testCustomScriptBoost() throws Exception {
assertThat(response.getHits().getAt(2).getFields().get("sNum1").getValues().get(0), equalTo(3.0));
}
- public void testDisallowExpensiveQueries() {
+ public void testDisallowExpensiveQueries() throws InterruptedException {
try {
assertAcked(prepareCreate("test-index").setMapping("num1", "type=double"));
int docCount = 10;
@@ -267,6 +269,7 @@ public void testDisallowExpensiveQueries() {
client().prepareIndex("test-index").setId("" + i).setSource("num1", i).get();
}
refresh();
+ indexRandomForConcurrentSearch("test-index");
// Execute with search.allow_expensive_queries = null => default value = false => success
Script script = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "doc['num1'].value > 1", Collections.emptyMap());
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollIT.java b/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollIT.java
index 0eee136acac69..b2b6409580061 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollIT.java
@@ -733,7 +733,7 @@ public void testInvalidScrollKeepAlive() throws IOException {
* Ensures that we always create and retain search contexts on every target shards for a scroll request
* regardless whether that query can be written to match_no_docs on some target shards or not.
*/
- public void testScrollRewrittenToMatchNoDocs() {
+ public void testScrollRewrittenToMatchNoDocs() throws InterruptedException {
final int numShards = randomIntBetween(3, 5);
assertAcked(
client().admin()
@@ -746,6 +746,7 @@ public void testScrollRewrittenToMatchNoDocs() {
client().prepareIndex("test").setId("2").setSource("created_date", "2020-01-02").get();
client().prepareIndex("test").setId("3").setSource("created_date", "2020-01-03").get();
client().admin().indices().prepareRefresh("test").get();
+ indexRandomForConcurrentSearch("test");
SearchResponse resp = null;
try {
int totalHits = 0;
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollWithFailingNodesIT.java b/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollWithFailingNodesIT.java
index f16b9a4d67b49..27002b844da1d 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollWithFailingNodesIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/scroll/SearchScrollWithFailingNodesIT.java
@@ -119,7 +119,7 @@ public void testScanScrollWithShardExceptions() throws Exception {
assertThat(numHits, equalTo(100L));
clearScroll("_all");
- internalCluster().stopRandomNonClusterManagerNode();
+ internalCluster().stopRandomDataNode();
searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(10).setScroll(TimeValue.timeValueMinutes(1)).get();
assertThat(searchResponse.getSuccessfulShards(), lessThan(searchResponse.getTotalShards()));
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/searchafter/SearchAfterIT.java b/server/src/internalClusterTest/java/org/opensearch/search/searchafter/SearchAfterIT.java
index 00ac574b8bd72..b99f66850e9e3 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/searchafter/SearchAfterIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/searchafter/SearchAfterIT.java
@@ -220,8 +220,8 @@ public void testPitWithSearchAfter() throws Exception {
.setPointInTime(new PointInTimeBuilder(pitResponse.getId()))
.get();
assertEquals(3, sr.getHits().getHits().length);
- /**
- * Add new data and assert PIT results remain the same and normal search results gets refreshed
+ /*
+ Add new data and assert PIT results remain the same and normal search results gets refreshed
*/
indexRandom(true, client().prepareIndex("test").setId("4").setSource("field1", 102));
sr = client().prepareSearch()
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/simple/ParameterizedSimpleSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/simple/ParameterizedSimpleSearchIT.java
deleted file mode 100644
index 719b75079da92..0000000000000
--- a/server/src/internalClusterTest/java/org/opensearch/search/simple/ParameterizedSimpleSearchIT.java
+++ /dev/null
@@ -1,608 +0,0 @@
-/*
- * SPDX-License-Identifier: Apache-2.0
- *
- * The OpenSearch Contributors require contributions made to
- * this file be licensed under the Apache-2.0 license or a
- * compatible open source license.
- */
-
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/*
- * Modifications Copyright OpenSearch Contributors. See
- * GitHub history for details.
- */
-
-package org.opensearch.search.simple;
-
-import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-
-import org.opensearch.action.index.IndexRequestBuilder;
-import org.opensearch.action.search.SearchPhaseExecutionException;
-import org.opensearch.action.search.SearchRequestBuilder;
-import org.opensearch.action.search.SearchResponse;
-import org.opensearch.action.support.WriteRequest.RefreshPolicy;
-import org.opensearch.common.settings.Settings;
-import org.opensearch.common.util.FeatureFlags;
-import org.opensearch.common.xcontent.XContentFactory;
-import org.opensearch.common.xcontent.json.JsonXContent;
-import org.opensearch.core.rest.RestStatus;
-import org.opensearch.core.xcontent.MediaTypeRegistry;
-import org.opensearch.core.xcontent.XContentParser;
-import org.opensearch.index.IndexSettings;
-import org.opensearch.index.mapper.MapperService;
-import org.opensearch.index.query.QueryBuilders;
-import org.opensearch.index.query.TermQueryBuilder;
-import org.opensearch.search.rescore.QueryRescorerBuilder;
-import org.opensearch.search.sort.SortOrder;
-import org.opensearch.test.ParameterizedOpenSearchIntegTestCase;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-
-import static org.opensearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
-import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
-import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS;
-import static org.opensearch.index.query.QueryBuilders.boolQuery;
-import static org.opensearch.index.query.QueryBuilders.matchAllQuery;
-import static org.opensearch.index.query.QueryBuilders.queryStringQuery;
-import static org.opensearch.index.query.QueryBuilders.rangeQuery;
-import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertFailures;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
-import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-
-public class ParameterizedSimpleSearchIT extends ParameterizedOpenSearchIntegTestCase {
-
- public ParameterizedSimpleSearchIT(Settings settings) {
- super(settings);
- }
-
- @ParametersFactory
- public static Collection parameters() {
- return Arrays.asList(
- new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), false).build() },
- new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build() }
- );
- }
-
- @Override
- protected Settings featureFlagSettings() {
- return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.CONCURRENT_SEGMENT_SEARCH, "true").build();
- }
-
- public void testSearchNullIndex() {
- expectThrows(
- NullPointerException.class,
- () -> client().prepareSearch((String) null).setQuery(QueryBuilders.termQuery("_id", "XXX1")).get()
- );
-
- expectThrows(
- NullPointerException.class,
- () -> client().prepareSearch((String[]) null).setQuery(QueryBuilders.termQuery("_id", "XXX1")).get()
- );
-
- }
-
- public void testSearchRandomPreference() throws InterruptedException, ExecutionException {
- createIndex("test");
- indexRandom(
- true,
- client().prepareIndex("test").setId("1").setSource("field", "value"),
- client().prepareIndex("test").setId("2").setSource("field", "value"),
- client().prepareIndex("test").setId("3").setSource("field", "value"),
- client().prepareIndex("test").setId("4").setSource("field", "value"),
- client().prepareIndex("test").setId("5").setSource("field", "value"),
- client().prepareIndex("test").setId("6").setSource("field", "value")
- );
-
- int iters = scaledRandomIntBetween(10, 20);
- for (int i = 0; i < iters; i++) {
- String randomPreference = randomUnicodeOfLengthBetween(0, 4);
- // randomPreference should not start with '_' (reserved for known preference types (e.g. _shards, _primary)
- while (randomPreference.startsWith("_")) {
- randomPreference = randomUnicodeOfLengthBetween(0, 4);
- }
- // id is not indexed, but lets see that we automatically convert to
- SearchResponse searchResponse = client().prepareSearch()
- .setQuery(QueryBuilders.matchAllQuery())
- .setPreference(randomPreference)
- .get();
- assertHitCount(searchResponse, 6L);
-
- }
- }
-
- public void testSimpleIp() throws Exception {
- createIndex("test");
-
- client().admin()
- .indices()
- .preparePutMapping("test")
- .setSource(
- XContentFactory.jsonBuilder()
- .startObject()
- .startObject(MapperService.SINGLE_MAPPING_NAME)
- .startObject("properties")
- .startObject("from")
- .field("type", "ip")
- .endObject()
- .startObject("to")
- .field("type", "ip")
- .endObject()
- .endObject()
- .endObject()
- .endObject()
- )
- .get();
-
- client().prepareIndex("test").setId("1").setSource("from", "192.168.0.5", "to", "192.168.0.10").setRefreshPolicy(IMMEDIATE).get();
-
- SearchResponse search = client().prepareSearch()
- .setQuery(boolQuery().must(rangeQuery("from").lte("192.168.0.7")).must(rangeQuery("to").gte("192.168.0.7")))
- .get();
-
- assertHitCount(search, 1L);
- }
-
- public void testIpCidr() throws Exception {
- createIndex("test");
-
- client().admin()
- .indices()
- .preparePutMapping("test")
- .setSource(
- XContentFactory.jsonBuilder()
- .startObject()
- .startObject(MapperService.SINGLE_MAPPING_NAME)
- .startObject("properties")
- .startObject("ip")
- .field("type", "ip")
- .endObject()
- .endObject()
- .endObject()
- .endObject()
- )
- .get();
- ensureGreen();
-
- client().prepareIndex("test").setId("1").setSource("ip", "192.168.0.1").get();
- client().prepareIndex("test").setId("2").setSource("ip", "192.168.0.2").get();
- client().prepareIndex("test").setId("3").setSource("ip", "192.168.0.3").get();
- client().prepareIndex("test").setId("4").setSource("ip", "192.168.1.4").get();
- client().prepareIndex("test").setId("5").setSource("ip", "2001:db8::ff00:42:8329").get();
- refresh();
-
- SearchResponse search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.0.1"))).get();
- assertHitCount(search, 1L);
-
- search = client().prepareSearch().setQuery(queryStringQuery("ip: 192.168.0.1")).get();
- assertHitCount(search, 1L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.0.1/32"))).get();
- assertHitCount(search, 1L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.0.0/24"))).get();
- assertHitCount(search, 3L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.0.0.0/8"))).get();
- assertHitCount(search, 4L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "0.0.0.0/0"))).get();
- assertHitCount(search, 4L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "2001:db8::ff00:42:8329/128"))).get();
- assertHitCount(search, 1L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "2001:db8::/64"))).get();
- assertHitCount(search, 1L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "::/0"))).get();
- assertHitCount(search, 5L);
-
- search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "192.168.1.5/32"))).get();
- assertHitCount(search, 0L);
-
- assertFailures(
- client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "0/0/0/0/0"))),
- RestStatus.BAD_REQUEST,
- containsString("Expected [ip/prefix] but was [0/0/0/0/0]")
- );
- }
-
- public void testSimpleId() {
- createIndex("test");
-
- client().prepareIndex("test").setId("XXX1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
- // id is not indexed, but lets see that we automatically convert to
- SearchResponse searchResponse = client().prepareSearch().setQuery(QueryBuilders.termQuery("_id", "XXX1")).get();
- assertHitCount(searchResponse, 1L);
-
- searchResponse = client().prepareSearch().setQuery(QueryBuilders.queryStringQuery("_id:XXX1")).get();
- assertHitCount(searchResponse, 1L);
- }
-
- public void testSimpleDateRange() throws Exception {
- createIndex("test");
- client().prepareIndex("test").setId("1").setSource("field", "2010-01-05T02:00").get();
- client().prepareIndex("test").setId("2").setSource("field", "2010-01-06T02:00").get();
- ensureGreen();
- refresh();
- SearchResponse searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.rangeQuery("field").gte("2010-01-03||+2d").lte("2010-01-04||+2d/d"))
- .get();
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, 2L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.rangeQuery("field").gte("2010-01-05T02:00").lte("2010-01-06T02:00"))
- .get();
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, 2L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.rangeQuery("field").gte("2010-01-05T02:00").lt("2010-01-06T02:00"))
- .get();
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, 1L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.rangeQuery("field").gt("2010-01-05T02:00").lt("2010-01-06T02:00"))
- .get();
- assertNoFailures(searchResponse);
- assertHitCount(searchResponse, 0L);
-
- searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.queryStringQuery("field:[2010-01-03||+2d TO 2010-01-04||+2d/d]"))
- .get();
- assertHitCount(searchResponse, 2L);
- }
-
- // TODO: combine this test with SimpleSearchIT.testSimpleTerminateAfterCount after
- // https://github.com/opensearch-project/OpenSearch/issues/8371
- public void testSimpleTerminateAfterCountWithSizeAndTrackHits() throws Exception {
- prepareCreate("test").setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)).get();
- ensureGreen();
- int numDocs = randomIntBetween(15, 29);
- List docbuilders = new ArrayList<>(numDocs);
-
- for (int i = 1; i <= numDocs; i++) {
- String id = String.valueOf(i);
- docbuilders.add(client().prepareIndex("test").setId(id).setSource("field", i));
- }
-
- indexRandom(true, docbuilders);
- ensureGreen();
- refresh();
-
- SearchResponse searchResponse;
- searchResponse = client().prepareSearch("test")
- .setQuery(QueryBuilders.matchAllQuery())
- .setTerminateAfter(numDocs)
- .setSize(0)
- .setTrackTotalHits(true)
- .get();
- assertEquals(0, searchResponse.getFailedShards());
- }
-
- public void testSimpleIndexSortEarlyTerminate() throws Exception {
- prepareCreate("test").setSettings(
- Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0).put("index.sort.field", "rank")
- ).setMapping("rank", "type=integer").get();
- ensureGreen();
- int max = randomIntBetween(3, 29);
- List docbuilders = new ArrayList<>(max);
-
- for (int i = max - 1; i >= 0; i--) {
- String id = String.valueOf(i);
- docbuilders.add(client().prepareIndex("test").setId(id).setSource("rank", i));
- }
-
- indexRandom(true, docbuilders);
- ensureGreen();
- refresh();
-
- SearchResponse searchResponse;
- for (int i = 1; i < max; i++) {
- searchResponse = client().prepareSearch("test")
- .addDocValueField("rank")
- .setTrackTotalHits(false)
- .addSort("rank", SortOrder.ASC)
- .setSize(i)
- .get();
- assertNull(searchResponse.getHits().getTotalHits());
- for (int j = 0; j < i; j++) {
- assertThat(searchResponse.getHits().getAt(j).field("rank").getValue(), equalTo((long) j));
- }
- }
- }
-
- public void testInsaneFromAndSize() throws Exception {
- createIndex("idx");
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertWindowFails(client().prepareSearch("idx").setFrom(Integer.MAX_VALUE));
- assertWindowFails(client().prepareSearch("idx").setSize(Integer.MAX_VALUE));
- }
-
- public void testTooLargeFromAndSize() throws Exception {
- createIndex("idx");
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertWindowFails(client().prepareSearch("idx").setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY)));
- assertWindowFails(client().prepareSearch("idx").setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) + 1));
- assertWindowFails(
- client().prepareSearch("idx")
- .setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- .setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- );
- }
-
- public void testLargeFromAndSizeSucceeds() throws Exception {
- createIndex("idx");
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(client().prepareSearch("idx").setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) - 10).get(), 1);
- assertHitCount(client().prepareSearch("idx").setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY)).get(), 1);
- assertHitCount(
- client().prepareSearch("idx")
- .setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) / 2)
- .setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) / 2 - 1)
- .get(),
- 1
- );
- }
-
- public void testTooLargeFromAndSizeOkBySetting() throws Exception {
- prepareCreate("idx").setSettings(
- Settings.builder()
- .put(IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey(), IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 2)
- ).get();
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(client().prepareSearch("idx").setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY)).get(), 1);
- assertHitCount(client().prepareSearch("idx").setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) + 1).get(), 1);
- assertHitCount(
- client().prepareSearch("idx")
- .setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- .setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- .get(),
- 1
- );
- }
-
- public void testTooLargeFromAndSizeOkByDynamicSetting() throws Exception {
- createIndex("idx");
- assertAcked(
- client().admin()
- .indices()
- .prepareUpdateSettings("idx")
- .setSettings(
- Settings.builder()
- .put(
- IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey(),
- IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 2
- )
- )
- .get()
- );
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(client().prepareSearch("idx").setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY)).get(), 1);
- assertHitCount(client().prepareSearch("idx").setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) + 1).get(), 1);
- assertHitCount(
- client().prepareSearch("idx")
- .setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- .setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY))
- .get(),
- 1
- );
- }
-
- public void testTooLargeFromAndSizeBackwardsCompatibilityRecommendation() throws Exception {
- prepareCreate("idx").setSettings(Settings.builder().put(IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey(), Integer.MAX_VALUE)).get();
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(client().prepareSearch("idx").setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 10).get(), 1);
- assertHitCount(client().prepareSearch("idx").setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 10).get(), 1);
- assertHitCount(
- client().prepareSearch("idx")
- .setSize(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 10)
- .setFrom(IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY) * 10)
- .get(),
- 1
- );
- }
-
- public void testTooLargeRescoreWindow() throws Exception {
- createIndex("idx");
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertRescoreWindowFails(Integer.MAX_VALUE);
- assertRescoreWindowFails(IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY) + 1);
- }
-
- public void testTooLargeRescoreOkBySetting() throws Exception {
- int defaultMaxWindow = IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY);
- prepareCreate("idx").setSettings(Settings.builder().put(IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey(), defaultMaxWindow * 2))
- .get();
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(
- client().prepareSearch("idx").addRescorer(new QueryRescorerBuilder(matchAllQuery()).windowSize(defaultMaxWindow + 1)).get(),
- 1
- );
- }
-
- public void testTooLargeRescoreOkByResultWindowSetting() throws Exception {
- int defaultMaxWindow = IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY);
- prepareCreate("idx").setSettings(
- Settings.builder()
- .put(
- IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey(), // Note that this is the RESULT window.
- defaultMaxWindow * 2
- )
- ).get();
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(
- client().prepareSearch("idx").addRescorer(new QueryRescorerBuilder(matchAllQuery()).windowSize(defaultMaxWindow + 1)).get(),
- 1
- );
- }
-
- public void testTooLargeRescoreOkByDynamicSetting() throws Exception {
- int defaultMaxWindow = IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY);
- createIndex("idx");
- assertAcked(
- client().admin()
- .indices()
- .prepareUpdateSettings("idx")
- .setSettings(Settings.builder().put(IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey(), defaultMaxWindow * 2))
- .get()
- );
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(
- client().prepareSearch("idx").addRescorer(new QueryRescorerBuilder(matchAllQuery()).windowSize(defaultMaxWindow + 1)).get(),
- 1
- );
- }
-
- public void testTooLargeRescoreOkByDynamicResultWindowSetting() throws Exception {
- int defaultMaxWindow = IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY);
- createIndex("idx");
- assertAcked(
- client().admin()
- .indices()
- .prepareUpdateSettings("idx")
- .setSettings(
- // Note that this is the RESULT window
- Settings.builder().put(IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey(), defaultMaxWindow * 2)
- )
- .get()
- );
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- assertHitCount(
- client().prepareSearch("idx").addRescorer(new QueryRescorerBuilder(matchAllQuery()).windowSize(defaultMaxWindow + 1)).get(),
- 1
- );
- }
-
- public void testQueryNumericFieldWithRegex() throws Exception {
- assertAcked(prepareCreate("idx").setMapping("num", "type=integer"));
- ensureGreen("idx");
-
- try {
- client().prepareSearch("idx").setQuery(QueryBuilders.regexpQuery("num", "34")).get();
- fail("SearchPhaseExecutionException should have been thrown");
- } catch (SearchPhaseExecutionException ex) {
- assertThat(ex.getRootCause().getMessage(), containsString("Can only use regexp queries on keyword and text fields"));
- }
- }
-
- public void testTermQueryBigInt() throws Exception {
- prepareCreate("idx").setMapping("field", "type=keyword").get();
- ensureGreen("idx");
-
- client().prepareIndex("idx")
- .setId("1")
- .setSource("{\"field\" : 80315953321748200608 }", MediaTypeRegistry.JSON)
- .setRefreshPolicy(RefreshPolicy.IMMEDIATE)
- .get();
-
- String queryJson = "{ \"field\" : { \"value\" : 80315953321748200608 } }";
- XContentParser parser = createParser(JsonXContent.jsonXContent, queryJson);
- parser.nextToken();
- TermQueryBuilder query = TermQueryBuilder.fromXContent(parser);
- SearchResponse searchResponse = client().prepareSearch("idx").setQuery(query).get();
- assertEquals(1, searchResponse.getHits().getTotalHits().value);
- }
-
- public void testTooLongRegexInRegexpQuery() throws Exception {
- createIndex("idx");
- indexRandom(true, client().prepareIndex("idx").setSource("{}", MediaTypeRegistry.JSON));
-
- int defaultMaxRegexLength = IndexSettings.MAX_REGEX_LENGTH_SETTING.get(Settings.EMPTY);
- StringBuilder regexp = new StringBuilder(defaultMaxRegexLength);
- while (regexp.length() <= defaultMaxRegexLength) {
- regexp.append("]\\r\\\\]|\\\\.)*\\](?:(?:\\r\\n)?[\\t])*))*(?:,@(?:(?:\\r\\n)?[ \\t])*(?:[^()<>@,;:\\\\\".\\");
- }
- SearchPhaseExecutionException e = expectThrows(
- SearchPhaseExecutionException.class,
- () -> client().prepareSearch("idx").setQuery(QueryBuilders.regexpQuery("num", regexp.toString())).get()
- );
- assertThat(
- e.getRootCause().getMessage(),
- containsString(
- "The length of regex ["
- + regexp.length()
- + "] used in the Regexp Query request has exceeded "
- + "the allowed maximum of ["
- + defaultMaxRegexLength
- + "]. "
- + "This maximum can be set by changing the ["
- + IndexSettings.MAX_REGEX_LENGTH_SETTING.getKey()
- + "] index level setting."
- )
- );
- }
-
- private void assertWindowFails(SearchRequestBuilder search) {
- SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () -> search.get());
- assertThat(
- e.toString(),
- containsString(
- "Result window is too large, from + size must be less than or equal to: ["
- + IndexSettings.MAX_RESULT_WINDOW_SETTING.get(Settings.EMPTY)
- )
- );
- assertThat(e.toString(), containsString("See the scroll api for a more efficient way to request large data sets"));
- }
-
- private void assertRescoreWindowFails(int windowSize) {
- SearchRequestBuilder search = client().prepareSearch("idx")
- .addRescorer(new QueryRescorerBuilder(matchAllQuery()).windowSize(windowSize));
- SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () -> search.get());
- assertThat(
- e.toString(),
- containsString(
- "Rescore window ["
- + windowSize
- + "] is too large. It must "
- + "be less than ["
- + IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY)
- )
- );
- assertThat(
- e.toString(),
- containsString(
- "This limit can be set by changing the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey() + "] index level setting."
- )
- );
- }
-}
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/simple/SimpleSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/simple/SimpleSearchIT.java
index 67e460653245e..7aae41d939cac 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/simple/SimpleSearchIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/simple/SimpleSearchIT.java
@@ -6,28 +6,286 @@
* compatible open source license.
*/
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
package org.opensearch.search.simple;
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+
import org.opensearch.action.index.IndexRequestBuilder;
+import org.opensearch.action.search.SearchPhaseExecutionException;
+import org.opensearch.action.search.SearchRequestBuilder;
import org.opensearch.action.search.SearchResponse;
+import org.opensearch.action.support.WriteRequest.RefreshPolicy;
import org.opensearch.common.settings.Settings;
+import org.opensearch.common.util.FeatureFlags;
+import org.opensearch.common.xcontent.XContentFactory;
+import org.opensearch.common.xcontent.json.JsonXContent;
+import org.opensearch.core.rest.RestStatus;
+import org.opensearch.core.xcontent.MediaTypeRegistry;
+import org.opensearch.core.xcontent.XContentParser;
+import org.opensearch.index.IndexSettings;
+import org.opensearch.index.mapper.MapperService;
import org.opensearch.index.query.QueryBuilders;
-import org.opensearch.test.OpenSearchIntegTestCase;
+import org.opensearch.index.query.TermQueryBuilder;
+import org.opensearch.search.rescore.QueryRescorerBuilder;
+import org.opensearch.search.sort.SortOrder;
+import org.opensearch.test.ParameterizedOpenSearchIntegTestCase;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
import java.util.List;
+import java.util.concurrent.ExecutionException;
+import static org.opensearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS;
+import static org.opensearch.index.query.QueryBuilders.boolQuery;
+import static org.opensearch.index.query.QueryBuilders.matchAllQuery;
+import static org.opensearch.index.query.QueryBuilders.queryStringQuery;
+import static org.opensearch.index.query.QueryBuilders.rangeQuery;
+import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertFailures;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount;
+import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.apache.lucene.search.TotalHits.Relation.EQUAL_TO;
+import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
+
+public class SimpleSearchIT extends ParameterizedOpenSearchIntegTestCase {
+
+ public SimpleSearchIT(Settings settings) {
+ super(settings);
+ }
+
+ @ParametersFactory
+ public static Collection