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 a87c060dcc60a..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
@@ -81,7 +81,6 @@
import org.opensearch.common.io.InputStreamContainer;
import org.opensearch.core.action.ActionListener;
import org.opensearch.core.common.unit.ByteSizeUnit;
-import org.opensearch.repositories.s3.async.AsyncTransferManager;
import org.opensearch.test.OpenSearchTestCase;
import java.io.ByteArrayInputStream;
@@ -100,7 +99,6 @@
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -278,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();
@@ -299,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));
@@ -313,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));
}
@@ -919,7 +924,7 @@ public void testListBlobsByPrefixInLexicographicOrderWithLimitGreaterThanNumberO
testListBlobsByPrefixInLexicographicOrder(12, 2, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC);
}
- public void testReadBlobAsync() throws Exception {
+ public void testReadBlobAsyncMultiPart() throws Exception {
final String bucketName = randomAlphaOfLengthBetween(1, 10);
final String blobName = randomAlphaOfLengthBetween(1, 10);
final String checksum = randomAlphaOfLength(10);
@@ -930,13 +935,9 @@ public void testReadBlobAsync() throws Exception {
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
- AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, null)
- );
- final AsyncTransferManager asyncTransferManager = new AsyncTransferManager(
- 10000L,
- mock(ExecutorService.class),
- mock(ExecutorService.class)
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
);
+
final S3BlobStore blobStore = mock(S3BlobStore.class);
final BlobPath blobPath = new BlobPath();
@@ -944,7 +945,6 @@ public void testReadBlobAsync() throws Exception {
when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher());
when(blobStore.serverSideEncryption()).thenReturn(false);
when(blobStore.asyncClientReference()).thenReturn(amazonAsyncS3Reference);
- when(blobStore.getAsyncTransferManager()).thenReturn(asyncTransferManager);
CompletableFuture getObjectAttributesResponseCompletableFuture = new CompletableFuture<>();
getObjectAttributesResponseCompletableFuture.complete(
@@ -976,7 +976,7 @@ public void testReadBlobAsync() 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());
@@ -984,6 +984,60 @@ public void testReadBlobAsync() throws Exception {
}
}
+ public void testReadBlobAsyncSinglePart() throws Exception {
+ final String bucketName = randomAlphaOfLengthBetween(1, 10);
+ final String blobName = randomAlphaOfLengthBetween(1, 10);
+ final String checksum = randomAlphaOfLength(10);
+
+ final int objectSize = 100;
+
+ 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().checksumCRC32(checksum).build())
+ .objectSize((long) objectSize)
+ .build()
+ );
+ when(s3AsyncClient.getObjectAttributes(any(GetObjectAttributesRequest.class))).thenReturn(
+ getObjectAttributesResponseCompletableFuture
+ );
+
+ mockObjectResponse(s3AsyncClient, bucketName, blobName, objectSize);
+
+ 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(1, readContextActionListener.getResponseCount());
+ assertEquals(0, readContextActionListener.getFailureCount());
+ ReadContext readContext = readContextActionListener.getResponse();
+ assertEquals(1, readContext.getNumberOfParts());
+ assertEquals(checksum, readContext.getBlobChecksum());
+ assertEquals(objectSize, readContext.getBlobSize());
+
+ InputStreamContainer inputStreamContainer = readContext.getPartStreams().stream().findFirst().get().get().join();
+ assertEquals(objectSize, inputStreamContainer.getContentLength());
+ assertEquals(0, inputStreamContainer.getOffset());
+ assertEquals(objectSize, inputStreamContainer.getInputStream().readAllBytes().length);
+
+ }
+
public void testReadBlobAsyncFailure() throws Exception {
final String bucketName = randomAlphaOfLengthBetween(1, 10);
final String blobName = randomAlphaOfLengthBetween(1, 10);
@@ -994,13 +1048,9 @@ public void testReadBlobAsyncFailure() throws Exception {
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference amazonAsyncS3Reference = new AmazonAsyncS3Reference(
- AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, null)
- );
- final AsyncTransferManager asyncTransferManager = new AsyncTransferManager(
- 10000L,
- mock(ExecutorService.class),
- mock(ExecutorService.class)
+ AmazonAsyncS3WithCredentials.create(s3AsyncClient, s3AsyncClient, s3AsyncClient, null)
);
+
final S3BlobStore blobStore = mock(S3BlobStore.class);
final BlobPath blobPath = new BlobPath();
@@ -1008,7 +1058,6 @@ public void testReadBlobAsyncFailure() throws Exception {
when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher());
when(blobStore.serverSideEncryption()).thenReturn(false);
when(blobStore.asyncClientReference()).thenReturn(amazonAsyncS3Reference);
- when(blobStore.getAsyncTransferManager()).thenReturn(asyncTransferManager);
CompletableFuture getObjectAttributesResponseCompletableFuture = new CompletableFuture<>();
getObjectAttributesResponseCompletableFuture.complete(
@@ -1032,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;
@@ -1071,7 +1165,7 @@ public void testGetBlobPartInputStream() throws Exception {
final String blobName = randomAlphaOfLengthBetween(1, 10);
final String bucketName = randomAlphaOfLengthBetween(1, 10);
final long contentLength = 10L;
- final String contentRange = "bytes 0-10/100";
+ final String contentRange = "bytes 10-20/100";
final InputStream inputStream = ResponseInputStream.nullInputStream();
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
@@ -1095,9 +1189,17 @@ public void testGetBlobPartInputStream() throws Exception {
)
).thenReturn(getObjectPartResponse);
+ // Header based offset in case of a multi part object request
InputStreamContainer inputStreamContainer = blobContainer.getBlobPartInputStreamContainer(s3AsyncClient, bucketName, blobName, 0)
.get();
+ assertEquals(10, inputStreamContainer.getOffset());
+ assertEquals(contentLength, inputStreamContainer.getContentLength());
+ assertEquals(inputStream.available(), inputStreamContainer.getInputStream().available());
+
+ // 0 offset in case of a single part object request
+ inputStreamContainer = blobContainer.getBlobPartInputStreamContainer(s3AsyncClient, bucketName, blobName, null).get();
+
assertEquals(0, inputStreamContainer.getOffset());
assertEquals(contentLength, inputStreamContainer.getContentLength());
assertEquals(inputStream.available(), inputStreamContainer.getInputStream().available());
@@ -1108,28 +1210,65 @@ public void testTransformResponseToInputStreamContainer() throws Exception {
final long contentLength = 10L;
final InputStream inputStream = ResponseInputStream.nullInputStream();
- final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
-
GetObjectResponse getObjectResponse = GetObjectResponse.builder().contentLength(contentLength).build();
+ // Exception when content range absent for multipart object
ResponseInputStream responseInputStreamNoRange = new ResponseInputStream<>(getObjectResponse, inputStream);
- assertThrows(SdkException.class, () -> S3BlobContainer.transformResponseToInputStreamContainer(responseInputStreamNoRange));
+ assertThrows(SdkException.class, () -> S3BlobContainer.transformResponseToInputStreamContainer(responseInputStreamNoRange, true));
+ // No exception when content range absent for single part object
+ ResponseInputStream responseInputStreamNoRangeSinglePart = new ResponseInputStream<>(
+ getObjectResponse,
+ inputStream
+ );
+ InputStreamContainer inputStreamContainer = S3BlobContainer.transformResponseToInputStreamContainer(
+ responseInputStreamNoRangeSinglePart,
+ false
+ );
+ assertEquals(contentLength, inputStreamContainer.getContentLength());
+ assertEquals(0, inputStreamContainer.getOffset());
+
+ // Exception when length is absent
getObjectResponse = GetObjectResponse.builder().contentRange(contentRange).build();
ResponseInputStream responseInputStreamNoContentLength = new ResponseInputStream<>(
getObjectResponse,
inputStream
);
- assertThrows(SdkException.class, () -> S3BlobContainer.transformResponseToInputStreamContainer(responseInputStreamNoContentLength));
+ assertThrows(
+ SdkException.class,
+ () -> S3BlobContainer.transformResponseToInputStreamContainer(responseInputStreamNoContentLength, true)
+ );
+ // No exception when range and length both are present
getObjectResponse = GetObjectResponse.builder().contentRange(contentRange).contentLength(contentLength).build();
ResponseInputStream responseInputStream = new ResponseInputStream<>(getObjectResponse, inputStream);
- InputStreamContainer inputStreamContainer = S3BlobContainer.transformResponseToInputStreamContainer(responseInputStream);
+ inputStreamContainer = S3BlobContainer.transformResponseToInputStreamContainer(responseInputStream, true);
assertEquals(contentLength, inputStreamContainer.getContentLength());
assertEquals(0, inputStreamContainer.getOffset());
assertEquals(inputStream.available(), inputStreamContainer.getInputStream().available());
}
+ private void mockObjectResponse(S3AsyncClient s3AsyncClient, String bucketName, String blobName, int objectSize) {
+
+ final InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(objectSize));
+
+ GetObjectResponse getObjectResponse = GetObjectResponse.builder().contentLength((long) objectSize).build();
+
+ CompletableFuture> getObjectPartResponse = new CompletableFuture<>();
+ ResponseInputStream responseInputStream = new ResponseInputStream<>(getObjectResponse, inputStream);
+ getObjectPartResponse.complete(responseInputStream);
+
+ GetObjectRequest getObjectRequest = GetObjectRequest.builder().bucket(bucketName).key(blobName).build();
+
+ when(
+ s3AsyncClient.getObject(
+ eq(getObjectRequest),
+ ArgumentMatchers.>>any()
+ )
+ ).thenReturn(getObjectPartResponse);
+
+ }
+
private void mockObjectPartResponse(
S3AsyncClient s3AsyncClient,
String bucketName,
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/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt
new file mode 100644
index 0000000000000..e69de29bb2d1d
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/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/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/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/nodes.stats/11_indices_metrics.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/nodes.stats/11_indices_metrics.yml
index 3f79227ce64e8..784c7b52b18b4 100644
--- a/rest-api-spec/src/main/resources/rest-api-spec/test/nodes.stats/11_indices_metrics.yml
+++ b/rest-api-spec/src/main/resources/rest-api-spec/test/nodes.stats/11_indices_metrics.yml
@@ -141,8 +141,8 @@
---
"Metric - indexing doc_status":
- skip:
- version: " - 2.99.99"
- reason: "To be introduced in future release :: TODO: change if/when we backport to 2.x"
+ version: " - 2.10.99"
+ reason: "Doc Status Stats were introduced in v2.11.0"
- do:
nodes.info: {}
- set:
diff --git a/server/build.gradle b/server/build.gradle
index f6db3d53a0dcc..c56f9d5aa288f 100644
--- a/server/build.gradle
+++ b/server/build.gradle
@@ -364,7 +364,9 @@ 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'
)
}
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/server/licenses/lucene-analysis-common-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-analysis-common-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 4ac89f2e792d7..0000000000000
--- a/server/licenses/lucene-analysis-common-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8c82be3d997d781bb72d6d0eadade064dd2cd6db
\ No newline at end of file
diff --git a/server/licenses/lucene-analysis-common-9.8.0.jar.sha1 b/server/licenses/lucene-analysis-common-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..6ad304fa52c12
--- /dev/null
+++ b/server/licenses/lucene-analysis-common-9.8.0.jar.sha1
@@ -0,0 +1 @@
+36f0363325ca7bf62c180160d1ed5165c7c37795
\ No newline at end of file
diff --git a/server/licenses/lucene-backward-codecs-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-backward-codecs-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 624b5174a444f..0000000000000
--- a/server/licenses/lucene-backward-codecs-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-4c261d17c681c0d91171c67e192abfef59adea2e
\ No newline at end of file
diff --git a/server/licenses/lucene-backward-codecs-9.8.0.jar.sha1 b/server/licenses/lucene-backward-codecs-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..f104c4207d390
--- /dev/null
+++ b/server/licenses/lucene-backward-codecs-9.8.0.jar.sha1
@@ -0,0 +1 @@
+e98fb408028f40170e6d87c16422bfdc0bb2e392
\ No newline at end of file
diff --git a/server/licenses/lucene-core-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-core-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 70baf1270cd5d..0000000000000
--- a/server/licenses/lucene-core-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-d2f7fbc5b2c49ca777a169d579f41082a9a57cc7
\ No newline at end of file
diff --git a/server/licenses/lucene-core-9.8.0.jar.sha1 b/server/licenses/lucene-core-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..f9a3e2f3cbee6
--- /dev/null
+++ b/server/licenses/lucene-core-9.8.0.jar.sha1
@@ -0,0 +1 @@
+5e8421c5f8573bcf22e9265fc7e19469545a775a
\ No newline at end of file
diff --git a/server/licenses/lucene-grouping-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-grouping-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 20ddb9ae3ef27..0000000000000
--- a/server/licenses/lucene-grouping-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8d1cf3d6db43fad6630376ba59451f848f4d387c
\ No newline at end of file
diff --git a/server/licenses/lucene-grouping-9.8.0.jar.sha1 b/server/licenses/lucene-grouping-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..ab132121b2edc
--- /dev/null
+++ b/server/licenses/lucene-grouping-9.8.0.jar.sha1
@@ -0,0 +1 @@
+d39184518351178c404ed9669fc6cb6111f2288d
\ No newline at end of file
diff --git a/server/licenses/lucene-highlighter-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-highlighter-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index c3ad03ca53b13..0000000000000
--- a/server/licenses/lucene-highlighter-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-83ab97638bb5269f950d75bba5675d3cfb63f2fa
\ No newline at end of file
diff --git a/server/licenses/lucene-highlighter-9.8.0.jar.sha1 b/server/licenses/lucene-highlighter-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..c7cb678fb7b72
--- /dev/null
+++ b/server/licenses/lucene-highlighter-9.8.0.jar.sha1
@@ -0,0 +1 @@
+1ac38c8278dbd63dfab30744a41dd955a415a31c
\ No newline at end of file
diff --git a/server/licenses/lucene-join-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-join-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index c2a4c5334b314..0000000000000
--- a/server/licenses/lucene-join-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-97c26362151908dc892263edda3872abbacb71a8
\ No newline at end of file
diff --git a/server/licenses/lucene-join-9.8.0.jar.sha1 b/server/licenses/lucene-join-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..2b6cb8af4faf6
--- /dev/null
+++ b/server/licenses/lucene-join-9.8.0.jar.sha1
@@ -0,0 +1 @@
+3d64fc57bb6e718d906413a9f73c713e6d4d8bb0
\ No newline at end of file
diff --git a/server/licenses/lucene-memory-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-memory-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 32534d07e47dc..0000000000000
--- a/server/licenses/lucene-memory-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-8337eddc0dddd0d7dd50c5aa0d17e5e31592f9fa
\ No newline at end of file
diff --git a/server/licenses/lucene-memory-9.8.0.jar.sha1 b/server/licenses/lucene-memory-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..5fdfee401dd0a
--- /dev/null
+++ b/server/licenses/lucene-memory-9.8.0.jar.sha1
@@ -0,0 +1 @@
+5283ac71d6ccecb5e00c7b52df2faec012f2625a
\ No newline at end of file
diff --git a/server/licenses/lucene-misc-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-misc-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 7db245cc521c7..0000000000000
--- a/server/licenses/lucene-misc-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-a2e3fae930295f0e2b401effe04eafc25692a414
\ No newline at end of file
diff --git a/server/licenses/lucene-misc-9.8.0.jar.sha1 b/server/licenses/lucene-misc-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..cf815cba15862
--- /dev/null
+++ b/server/licenses/lucene-misc-9.8.0.jar.sha1
@@ -0,0 +1 @@
+9a57b049cf51a5e9c9c1909c420f645f1b6f9a54
\ No newline at end of file
diff --git a/server/licenses/lucene-queries-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-queries-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index d01a6d733196e..0000000000000
--- a/server/licenses/lucene-queries-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-e88d8a464e6cfa345b946c9c8822ba7ee2a9159f
\ No newline at end of file
diff --git a/server/licenses/lucene-queries-9.8.0.jar.sha1 b/server/licenses/lucene-queries-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..09f369ef18e12
--- /dev/null
+++ b/server/licenses/lucene-queries-9.8.0.jar.sha1
@@ -0,0 +1 @@
+628db4ef46f1c6a05145bdac1d1bc4ace6341b13
\ No newline at end of file
diff --git a/server/licenses/lucene-queryparser-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-queryparser-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index c7b9640bad170..0000000000000
--- a/server/licenses/lucene-queryparser-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-9905790675c01e8dc24f9a5e6b9b28b879c65a52
\ No newline at end of file
diff --git a/server/licenses/lucene-queryparser-9.8.0.jar.sha1 b/server/licenses/lucene-queryparser-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..2a42a8956b18b
--- /dev/null
+++ b/server/licenses/lucene-queryparser-9.8.0.jar.sha1
@@ -0,0 +1 @@
+982faf2bfa55542bf57fbadef54c19ac00f57cae
\ No newline at end of file
diff --git a/server/licenses/lucene-sandbox-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-sandbox-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index c4cd9e47624f8..0000000000000
--- a/server/licenses/lucene-sandbox-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-d6c8be427ec8ffc7e8233ffbf0d190d95a56cf14
\ No newline at end of file
diff --git a/server/licenses/lucene-sandbox-9.8.0.jar.sha1 b/server/licenses/lucene-sandbox-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..64a0b07f72d29
--- /dev/null
+++ b/server/licenses/lucene-sandbox-9.8.0.jar.sha1
@@ -0,0 +1 @@
+06493dbd14d02537716822254866a94458f4d842
\ No newline at end of file
diff --git a/server/licenses/lucene-spatial-extras-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-spatial-extras-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index dfee145d3ea26..0000000000000
--- a/server/licenses/lucene-spatial-extras-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-11716d61288feaa692593bf699affa8de2b564c4
\ No newline at end of file
diff --git a/server/licenses/lucene-spatial-extras-9.8.0.jar.sha1 b/server/licenses/lucene-spatial-extras-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..d1bcb0581435c
--- /dev/null
+++ b/server/licenses/lucene-spatial-extras-9.8.0.jar.sha1
@@ -0,0 +1 @@
+9d9a731822ad6eefa1ba288a0c158d478522f165
\ No newline at end of file
diff --git a/server/licenses/lucene-spatial3d-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-spatial3d-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index c7410086ba86c..0000000000000
--- a/server/licenses/lucene-spatial3d-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-3a888e06c0535403b9e58a8dcddeb5e6513a4930
\ No newline at end of file
diff --git a/server/licenses/lucene-spatial3d-9.8.0.jar.sha1 b/server/licenses/lucene-spatial3d-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..d17459cc569a9
--- /dev/null
+++ b/server/licenses/lucene-spatial3d-9.8.0.jar.sha1
@@ -0,0 +1 @@
+ce752a52b2d4eac90633c7df7982e29504f99e76
\ No newline at end of file
diff --git a/server/licenses/lucene-suggest-9.8.0-snapshot-95cdd2e.jar.sha1 b/server/licenses/lucene-suggest-9.8.0-snapshot-95cdd2e.jar.sha1
deleted file mode 100644
index 6d8d4205f4d02..0000000000000
--- a/server/licenses/lucene-suggest-9.8.0-snapshot-95cdd2e.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-52dfc8bf135ed29f5baf0a967c1bb63dedb9a069
\ No newline at end of file
diff --git a/server/licenses/lucene-suggest-9.8.0.jar.sha1 b/server/licenses/lucene-suggest-9.8.0.jar.sha1
new file mode 100644
index 0000000000000..ff47b87672d2c
--- /dev/null
+++ b/server/licenses/lucene-suggest-9.8.0.jar.sha1
@@ -0,0 +1 @@
+f977f96f2093b7fddea6b67caa2e1c5b10edebf6
\ 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/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..59eef3c06844b 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,53 @@ public void testFullClusterRestoreStaleDelete() throws Exception {
assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size());
- Map indexMetadataMap = remoteClusterStateService.getLatestIndexMetadata(
+ Map indexMetadataMap = remoteClusterStateService.getLatestMetadata(
cluster().getClusterName(),
getClusterState().metadata().clusterUUID()
- );
+ ).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
+ 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);
+
+ 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());
+ }
+
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/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/SegmentReplicationClusterSettingIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java
index a82fd8d845709..186a5ce39f131 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationClusterSettingIT.java
@@ -19,6 +19,7 @@
import org.opensearch.test.OpenSearchIntegTestCase;
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE;
+import static org.opensearch.indices.IndicesService.CLUSTER_RESTRICT_INDEX_REPLICATION_TYPE_SETTING;
import static org.opensearch.indices.IndicesService.CLUSTER_SETTING_REPLICATION_TYPE;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
@@ -123,4 +124,30 @@ public void testIndexReplicationSettingOverridesDocRepClusterSetting() throws Ex
assertEquals(indicesService.indexService(anotherIndex).getIndexSettings().isSegRepEnabled(), false);
}
+ public void testIndexReplicationTypeWhenRestrictSettingTrue() {
+ testRestrictIndexReplicationTypeSetting(true, randomFrom(ReplicationType.values()));
+ }
+
+ public void testIndexReplicationTypeWhenRestrictSettingFalse() {
+ testRestrictIndexReplicationTypeSetting(false, randomFrom(ReplicationType.values()));
+ }
+
+ private void testRestrictIndexReplicationTypeSetting(boolean setRestrict, ReplicationType replicationType) {
+ String expectedExceptionMsg =
+ "Validation Failed: 1: index setting [index.replication.type] is not allowed to be set as [cluster.restrict.index.replication_type=true];";
+ String clusterManagerName = internalCluster().startNode(
+ Settings.builder().put(CLUSTER_RESTRICT_INDEX_REPLICATION_TYPE_SETTING.getKey(), setRestrict).build()
+ );
+ internalCluster().startDataOnlyNodes(1);
+
+ // Test create index fails
+ Settings indexSettings = Settings.builder().put(indexSettings()).put(SETTING_REPLICATION_TYPE, replicationType).build();
+ if (setRestrict) {
+ IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> createIndex(INDEX_NAME, indexSettings));
+ assertEquals(expectedExceptionMsg, exception.getMessage());
+ } else {
+ createIndex(INDEX_NAME, indexSettings);
+ }
+ }
+
}
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..f48df082a25dc 100644
--- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java
@@ -1776,5 +1776,4 @@ public void testRealtimeTermVectorRequestsUnSuccessful() throws IOException {
assertThat(response.getIndex(), equalTo(INDEX_NAME));
}
-
}
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..0967acb37d3e8 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;
@@ -589,8 +589,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 +621,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..9e0b2a66467de 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
@@ -345,6 +355,90 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException {
assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 4);
}
+ 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);
+
+ // Make sure remote translog is empty
+ String indexUUID = client().admin()
+ .indices()
+ .prepareGetSettings(indexName1)
+ .get()
+ .getSetting(indexName1, 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");
+
+ try (
+ Stream translogMetadata = Files.list(remoteTranslogMetadataPath);
+ Stream translogData = Files.list(remoteTranslogDataPath)
+ ) {
+ assertTrue(translogData.count() > 0);
+ assertTrue(translogMetadata.count() > 0);
+ }
+
+ // 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 +516,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 +558,7 @@ 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()
- .cluster()
- .prepareRestoreSnapshot(snapshotRepoName, snapshotName1)
- .setWaitForCompletion(true)
- .setIndices(indexName1)
- .setRenamePattern(indexName1)
- .setRenameReplacement(restoredIndexName1)
- .get();
-
- assertTrue(restoreSnapshotResponse.getRestoreInfo().failedShards() > 0);
-
- ensureRed(restoredIndexName1);
-
- 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..bccca283ba772 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,7 +51,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 class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase {
protected static final String REPOSITORY_NAME = "test-remote-store-repo";
@@ -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..e9afd6d36bb87 100644
--- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java
@@ -8,23 +8,40 @@
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.util.Locale;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
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,47 +65,17 @@ 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
-
- if (validate) {
- // Step - 4 validation restore is successful.
- ensureGreen(INDEX_NAME);
- verifyRestoredData(indexStats, INDEX_NAME);
- }
- }
-
- private void restoreAndValidateFails(
- String clusterUUID,
- PlainActionFuture actionListener,
- Class extends Throwable> clazz,
- String errorSubString
- ) {
-
- 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 {
int shardCount = randomIntBetween(1, 2);
int replicaCount = 1;
@@ -106,10 +93,76 @@ public void testFullClusterRestore() throws Exception {
assert !Objects.equals(newClusterUUID, prevClusterUUID) : "cluster restart not successful. cluster uuid is same";
// Step - 3 Trigger full cluster restore and validate
- restoreAndValidate(prevClusterUUID, indexStats);
+ validateMetadata(List.of(INDEX_NAME));
+ verifyRedIndicesAndTriggerRestore(indexStats, INDEX_NAME, true);
+ }
+
+ /**
+ * 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;
+
+ // index some data to generate files in remote directory
+ Map indexStats = initialTestSetup(shardCount, replicaCount, dataNodeCount, 1);
+ String prevClusterUUID = clusterService().state().metadata().clusterUUID();
+
+ // 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";
+ 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,6 +177,7 @@ 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();
@@ -134,155 +188,234 @@ public void testFullClusterRestoreMultipleIndices() throws Exception {
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);
+ 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();
+ // 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";
+ 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();
+ // 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);
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 - 5 Trigger full cluster restore and validate fails
- PlainActionFuture future = PlainActionFuture.newFuture();
- restoreAndValidateFails(prevClusterUUID, future, IllegalStateException.class, "asdsa");
+ // Step - 3 Trigger full cluster restore and validate
+ // 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));
+
+ 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 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 resetShardLimits() {
- // Step - 5 Reset the cluster settings
+ 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/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..865dd670fbf68 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
@@ -221,6 +221,10 @@ public void testNestedDiversity() throws Exception {
}
public void testNestedSamples() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/10046",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
// Test samples nested under samples
int MAX_DOCS_PER_AUTHOR = 1;
int MAX_DOCS_PER_GENRE = 2;
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..2aad0d2d38901 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() {
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..42d91ac945662 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
@@ -3343,6 +3343,10 @@ public void testFiltersFunctionScoreQueryHighlight() throws Exception {
}
public void testHighlightQueryRewriteDatesWithNow() throws Exception {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/10434",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(
client().admin()
.indices()
diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fieldcaps/FieldCapabilitiesIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fieldcaps/FieldCapabilitiesIT.java
index f5d1b8234558e..6b95405b3ebd4 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/fieldcaps/FieldCapabilitiesIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/fieldcaps/FieldCapabilitiesIT.java
@@ -244,6 +244,10 @@ public void testWithIndexAlias() {
}
public void testWithIndexFilter() throws InterruptedException {
+ assumeFalse(
+ "Concurrent search case muted pending fix: https://github.com/opensearch-project/OpenSearch/issues/10433",
+ internalCluster().clusterService().getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING)
+ );
assertAcked(prepareCreate("index-1").setMapping("timestamp", "type=date", "field1", "type=keyword"));
assertAcked(prepareCreate("index-2").setMapping("timestamp", "type=date", "field1", "type=long"));
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..4a178e7066846 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());
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/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 parameters() {
+ return Arrays.asList(
+ new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build() },
+ new Object[] { Settings.builder().put(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), false).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);
-public class SimpleSearchIT extends OpenSearchIntegTestCase {
+ search = client().prepareSearch().setQuery(boolQuery().must(QueryBuilders.termQuery("ip", "2001:db8::ff00:42:8329/128"))).get();
+ assertHitCount(search, 1L);
- // TODO: Move this test to ParameterizedSimpleSearchIT after https://github.com/opensearch-project/OpenSearch/issues/8371
- public void testSimpleTerminateAfterCount() throws Exception {
+ 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);
+ }
+
+ public void dotestSimpleTerminateAfterCountWithSize(int size, int max) throws Exception {
prepareCreate("test").setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)).get();
ensureGreen();
- int max = randomIntBetween(3, 29);
List docbuilders = new ArrayList<>(max);
for (int i = 1; i <= max; i++) {
@@ -44,9 +302,12 @@ public void testSimpleTerminateAfterCount() throws Exception {
searchResponse = client().prepareSearch("test")
.setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(max))
.setTerminateAfter(i)
+ .setSize(size)
+ .setTrackTotalHits(true)
.get();
assertHitCount(searchResponse, i);
assertTrue(searchResponse.isTerminatedEarly());
+ assertEquals(Math.min(i, size), searchResponse.getHits().getHits().length);
}
searchResponse = client().prepareSearch("test")
@@ -57,4 +318,397 @@ public void testSimpleTerminateAfterCount() throws Exception {
assertHitCount(searchResponse, max);
assertFalse(searchResponse.isTerminatedEarly());
}
+
+ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/10435")
+ public void testSimpleTerminateAfterCountSize0() throws Exception {
+ int max = randomIntBetween(3, 29);
+ dotestSimpleTerminateAfterCountWithSize(0, max);
+ }
+
+ public void testSimpleTerminateAfterCountRandomSize() throws Exception {
+ int max = randomIntBetween(3, 29);
+ dotestSimpleTerminateAfterCountWithSize(randomIntBetween(1, max), max);
+ }
+
+ public void doTestSimpleTerminateAfterTrackTotalHitsUpTo(int size) throws Exception {
+ prepareCreate("test").setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)).get();
+ ensureGreen();
+ int numDocs = 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.rangeQuery("field").gte(1).lte(numDocs))
+ .setTerminateAfter(10)
+ .setSize(size)
+ .setTrackTotalHitsUpTo(5)
+ .get();
+ assertTrue(searchResponse.isTerminatedEarly());
+ assertEquals(5, searchResponse.getHits().getTotalHits().value);
+ assertEquals(GREATER_THAN_OR_EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+
+ searchResponse = client().prepareSearch("test")
+ .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(numDocs))
+ .setTerminateAfter(5)
+ .setSize(size)
+ .setTrackTotalHitsUpTo(10)
+ .get();
+ assertTrue(searchResponse.isTerminatedEarly());
+ assertEquals(5, searchResponse.getHits().getTotalHits().value);
+ assertEquals(EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+
+ searchResponse = client().prepareSearch("test")
+ .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(numDocs))
+ .setTerminateAfter(5)
+ .setSize(size)
+ .setTrackTotalHitsUpTo(5)
+ .get();
+ assertTrue(searchResponse.isTerminatedEarly());
+ assertEquals(5, searchResponse.getHits().getTotalHits().value);
+ assertEquals(EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+
+ searchResponse = client().prepareSearch("test")
+ .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(numDocs))
+ .setTerminateAfter(5)
+ .setSize(size)
+ .setTrackTotalHits(true)
+ .get();
+ assertTrue(searchResponse.isTerminatedEarly());
+ assertEquals(5, searchResponse.getHits().getTotalHits().value);
+ assertEquals(EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+
+ searchResponse = client().prepareSearch("test")
+ .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(numDocs))
+ .setTerminateAfter(numDocs * 2)
+ .setSize(size)
+ .setTrackTotalHits(true)
+ .get();
+ assertFalse(searchResponse.isTerminatedEarly());
+ assertEquals(numDocs, searchResponse.getHits().getTotalHits().value);
+ assertEquals(EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+
+ searchResponse = client().prepareSearch("test")
+ .setQuery(QueryBuilders.rangeQuery("field").gte(1).lte(numDocs))
+ .setSize(size)
+ .setTrackTotalHitsUpTo(5)
+ .get();
+ assertEquals(5, searchResponse.getHits().getTotalHits().value);
+ assertEquals(GREATER_THAN_OR_EQUAL_TO, searchResponse.getHits().getTotalHits().relation);
+ }
+
+ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/10435")
+ public void testSimpleTerminateAfterTrackTotalHitsUpToRandomSize() throws Exception {
+ doTestSimpleTerminateAfterTrackTotalHitsUpTo(0);
+ }
+
+ public void testSimpleTerminateAfterTrackTotalHitsUpToSize0() throws Exception {
+ doTestSimpleTerminateAfterTrackTotalHitsUpTo(randomIntBetween(1, 29));
+ }
+
+ 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/sort/GeoDistanceSortBuilderIT.java b/server/src/internalClusterTest/java/org/opensearch/search/sort/GeoDistanceSortBuilderIT.java
index 7880fc24fd846..1b8bd9694483d 100644
--- a/server/src/internalClusterTest/java/org/opensearch/search/sort/GeoDistanceSortBuilderIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/search/sort/GeoDistanceSortBuilderIT.java
@@ -91,16 +91,16 @@ protected boolean forbidPrivateIndexSettings() {
}
public void testManyToManyGeoPoints() throws ExecutionException, InterruptedException, IOException {
- /**
- * | q | d1 | d2
- * | | |
- * | | |
- * | | |
- * |2 o| x | x
- * | | |
- * |1 o| x | x
- * |___________________________
- * 1 2 3 4 5 6 7
+ /*
+ | q | d1 | d2
+ | | |
+ | | |
+ | | |
+ |2 o| x | x
+ | | |
+ |1 o| x | x
+ |___________________________
+ 1 2 3 4 5 6 7
*/
Version version = randomBoolean() ? Version.CURRENT : VersionUtils.randomIndexCompatibleVersion(random());
Settings settings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, version).build();
@@ -187,11 +187,10 @@ public void testManyToManyGeoPoints() throws ExecutionException, InterruptedExce
}
public void testSingeToManyAvgMedian() throws ExecutionException, InterruptedException, IOException {
- /**
- * q = (0, 0)
- *
- * d1 = (0, 1), (0, 4), (0, 10); so avg. distance is 5, median distance is 4
- * d2 = (0, 1), (0, 5), (0, 6); so avg. distance is 4, median distance is 5
+ /*
+ q = (0, 0)
+ d1 = (0, 1), (0, 4), (0, 10); so avg. distance is 5, median distance is 4
+ d2 = (0, 1), (0, 5), (0, 6); so avg. distance is 4, median distance is 5
*/
Version version = randomBoolean() ? Version.CURRENT : VersionUtils.randomIndexCompatibleVersion(random());
Settings settings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, version).build();
@@ -256,16 +255,17 @@ protected void createShuffeldJSONArray(XContentBuilder builder, GeoPoint[] point
}
public void testManyToManyGeoPointsWithDifferentFormats() throws ExecutionException, InterruptedException, IOException {
- /** q d1 d2
- * |4 o| x | x
- * | | |
- * |3 o| x | x
- * | | |
- * |2 o| x | x
- * | | |
- * |1 o|x |x
- * |______________________
- * 1 2 3 4 5 6
+ /*
+ q d1 d2
+ |4 o| x | x
+ | | |
+ |3 o| x | x
+ | | |
+ |2 o| x | x
+ | | |
+ |1 o|x |x
+ |______________________
+ 1 2 3 4 5 6
*/
Version version = randomBoolean() ? Version.CURRENT : VersionUtils.randomIndexCompatibleVersion(random());
Settings settings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, version).build();
diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/CloneSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/CloneSnapshotIT.java
index 066d82483ae91..83f93ab9ff6b5 100644
--- a/server/src/internalClusterTest/java/org/opensearch/snapshots/CloneSnapshotIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/CloneSnapshotIT.java
@@ -215,9 +215,6 @@ public void testShallowCloneNameAvailability() throws Exception {
final Path shallowSnapshotRepoPath = randomRepoPath();
createRepository(shallowSnapshotRepoName, "fs", snapshotRepoSettingsForShallowCopy(shallowSnapshotRepoPath));
- final Path remoteStoreRepoPath = randomRepoPath();
- createRepository(remoteStoreRepoName, "fs", remoteStoreRepoPath);
-
final String indexName = "index-1";
createIndexWithRandomDocs(indexName, randomIntBetween(5, 10));
diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java
index 5268d5ddf35af..91b0aa6438753 100644
--- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java
@@ -1884,7 +1884,7 @@ public void testCannotCreateSnapshotsWithSameName() throws Exception {
* This test ensures that when a shard is removed from a node (perhaps due to the node
* leaving the cluster, then returning), all snapshotting of that shard is aborted, so
* all Store references held onto by the snapshot are released.
- *
+ *
* See https://github.com/elastic/elasticsearch/issues/20876
*/
public void testSnapshotCanceledOnRemovedShard() throws Exception {
diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SystemRepositoryIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SystemRepositoryIT.java
new file mode 100644
index 0000000000000..f50fc691fb232
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SystemRepositoryIT.java
@@ -0,0 +1,71 @@
+/*
+ * 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.snapshots;
+
+import org.opensearch.client.Client;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.repositories.RepositoryException;
+import org.opensearch.repositories.fs.FsRepository;
+import org.opensearch.test.OpenSearchIntegTestCase;
+import org.junit.Before;
+
+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 SystemRepositoryIT extends AbstractSnapshotIntegTestCase {
+ protected Path absolutePath;
+ final String systemRepoName = "system-repo-name";
+
+ @Before
+ public void setup() {
+ absolutePath = randomRepoPath().toAbsolutePath();
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put(remoteStoreClusterSettings(systemRepoName, absolutePath))
+ .build();
+ }
+
+ public void testRestrictedSettingsCantBeUpdated() {
+ disableRepoConsistencyCheck("System repository is being used for the test");
+
+ internalCluster().startNode();
+ final Client client = client();
+ final Settings.Builder repoSettings = Settings.builder().put("location", randomRepoPath());
+
+ RepositoryException e = expectThrows(
+ RepositoryException.class,
+ () -> client.admin().cluster().preparePutRepository(systemRepoName).setType("mock").setSettings(repoSettings).get()
+ );
+ assertEquals(
+ e.getMessage(),
+ "[system-repo-name] trying to modify an unmodifiable attribute type of system "
+ + "repository from current value [fs] to new value [mock]"
+ );
+ }
+
+ public void testSystemRepositoryNonRestrictedSettingsCanBeUpdated() {
+ disableRepoConsistencyCheck("System repository is being used for the test");
+
+ internalCluster().startNode();
+ final Client client = client();
+ final Settings.Builder repoSettings = Settings.builder().put("location", absolutePath).put("chunk_size", new ByteSizeValue(20));
+
+ assertAcked(
+ client.admin().cluster().preparePutRepository(systemRepoName).setType(FsRepository.TYPE).setSettings(repoSettings).get()
+ );
+ }
+}
diff --git a/server/src/internalClusterTest/java/org/opensearch/update/UpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/update/UpdateIT.java
index 442268d513fc3..b46d27bafb2a5 100644
--- a/server/src/internalClusterTest/java/org/opensearch/update/UpdateIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/update/UpdateIT.java
@@ -50,7 +50,7 @@
import org.opensearch.common.xcontent.XContentFactory;
import org.opensearch.core.action.ActionListener;
import org.opensearch.core.rest.RestStatus;
-import org.opensearch.index.MergePolicyConfig;
+import org.opensearch.index.MergePolicyProvider;
import org.opensearch.index.engine.DocumentMissingException;
import org.opensearch.index.engine.VersionConflictEngineException;
import org.opensearch.plugins.Plugin;
@@ -669,7 +669,7 @@ public void run() {
public void testStressUpdateDeleteConcurrency() throws Exception {
// We create an index with merging disabled so that deletes don't get merged away
- assertAcked(prepareCreate("test").setSettings(Settings.builder().put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)));
+ assertAcked(prepareCreate("test").setSettings(Settings.builder().put(MergePolicyProvider.INDEX_MERGE_ENABLED, false)));
ensureGreen();
Script fieldIncScript = new Script(ScriptType.INLINE, UPDATE_SCRIPTS, FIELD_INC_SCRIPT, Collections.singletonMap("field", "field"));
diff --git a/server/src/internalClusterTest/java/org/opensearch/versioning/ConcurrentSeqNoVersioningIT.java b/server/src/internalClusterTest/java/org/opensearch/versioning/ConcurrentSeqNoVersioningIT.java
index c651689e21d3d..7f016caf22149 100644
--- a/server/src/internalClusterTest/java/org/opensearch/versioning/ConcurrentSeqNoVersioningIT.java
+++ b/server/src/internalClusterTest/java/org/opensearch/versioning/ConcurrentSeqNoVersioningIT.java
@@ -93,7 +93,7 @@
* provided the primaryTerm and seqNo still matches. The reason we cannot assume it will not take place after receiving the failure
* is that a request can fork into two because of retries on disconnect, and now race against itself. The retry might complete (and do a
* dirty or stale read) before the forked off request gets to execute, and that one might still subsequently succeed.
- *
+ *
* Such writes are not necessarily fully replicated and can be lost. There is no
* guarantee that the previous value did not have the specified primaryTerm and seqNo
*
CAS writes with other exceptions might or might not have taken place. If they have taken place, then after invocation but not
diff --git a/server/src/main/java/org/apache/lucene/index/OneMergeHelper.java b/server/src/main/java/org/apache/lucene/index/OneMergeHelper.java
index 8f67bb87b5c42..0ada02a09d157 100644
--- a/server/src/main/java/org/apache/lucene/index/OneMergeHelper.java
+++ b/server/src/main/java/org/apache/lucene/index/OneMergeHelper.java
@@ -59,8 +59,8 @@ public static double getMbPerSec(Thread thread, MergePolicy.OneMerge merge) {
* Returns total bytes written by this merge.
**/
public static long getTotalBytesWritten(Thread thread, MergePolicy.OneMerge merge) throws IOException {
- /**
- * TODO: The number of bytes written during the merge should be accessible in OneMerge.
+ /*
+ TODO: The number of bytes written during the merge should be accessible in OneMerge.
*/
if (thread instanceof ConcurrentMergeScheduler.MergeThread) {
return ((ConcurrentMergeScheduler.MergeThread) thread).rateLimiter.getTotalBytesWritten();
diff --git a/server/src/main/java/org/apache/lucene/queryparser/classic/XQueryParser.java b/server/src/main/java/org/apache/lucene/queryparser/classic/XQueryParser.java
index 9a8c295d60ec7..2f00ea69207a7 100644
--- a/server/src/main/java/org/apache/lucene/queryparser/classic/XQueryParser.java
+++ b/server/src/main/java/org/apache/lucene/queryparser/classic/XQueryParser.java
@@ -38,7 +38,7 @@
/**
* This class is just a workaround to make {@link QueryParser#handleBareFuzzy(String, Token, String)} accessible by sub-classes.
* It is needed for {@link QueryParser}s that need to override the parsing of the slop in a fuzzy query (e.g. word~2 , word~ ).
- *
+ *
* TODO: We should maybe rewrite this with the flexible query parser which matches the same syntax with more freedom.
*/
public class XQueryParser extends QueryParser {
diff --git a/server/src/main/java/org/apache/lucene/search/grouping/CollapsingTopDocsCollector.java b/server/src/main/java/org/apache/lucene/search/grouping/CollapsingTopDocsCollector.java
index e4c299ba572b1..9ca0491bc29f5 100644
--- a/server/src/main/java/org/apache/lucene/search/grouping/CollapsingTopDocsCollector.java
+++ b/server/src/main/java/org/apache/lucene/search/grouping/CollapsingTopDocsCollector.java
@@ -50,7 +50,7 @@
* A collector that groups documents based on field values and returns {@link CollapseTopFieldDocs}
* output. The collapsing is done in a single pass by selecting only the top sorted document per collapse key.
* The value used for the collapse key of each group can be found in {@link CollapseTopFieldDocs#collapseValues}.
- *
+ *
* TODO: If the sort is based on score we should propagate the mininum competitive score when orderedGroups
is full.
* This is safe for collapsing since the group sort
is the same as the query sort.
*/
diff --git a/server/src/main/java/org/apache/lucene/search/uhighlight/BoundedBreakIteratorScanner.java b/server/src/main/java/org/apache/lucene/search/uhighlight/BoundedBreakIteratorScanner.java
index 6fde39b16a59a..4edcdea42b53b 100644
--- a/server/src/main/java/org/apache/lucene/search/uhighlight/BoundedBreakIteratorScanner.java
+++ b/server/src/main/java/org/apache/lucene/search/uhighlight/BoundedBreakIteratorScanner.java
@@ -43,13 +43,13 @@
* If the {@link BreakIterator} cannot find a passage smaller than the maximum length,
* a secondary break iterator is used to re-split the passage at the first boundary after
* maximum length.
- *
+ *
* This is useful to split passages created by {@link BreakIterator}s like `sentence` that
* can create big outliers on semi-structured text.
- *
+ *
*
* WARNING: This break iterator is designed to work with the {@link UnifiedHighlighter}.
- *
+ *
* TODO: We should be able to create passages incrementally, starting from the offset of the first match and expanding or not
* depending on the offsets of subsequent matches. This is currently impossible because {@link FieldHighlighter} uses
* only the first matching offset to derive the start and end of each passage.
diff --git a/server/src/main/java/org/apache/lucene/util/packed/XPacked64.java b/server/src/main/java/org/apache/lucene/util/packed/XPacked64.java
index 4777b77cfbfed..9e9f6d1fd817d 100644
--- a/server/src/main/java/org/apache/lucene/util/packed/XPacked64.java
+++ b/server/src/main/java/org/apache/lucene/util/packed/XPacked64.java
@@ -38,8 +38,8 @@
/**
* Forked from Lucene 8.x; removed in Lucene 9.0
*
- * @todo further investigate a better alternative
- *
+ * TODO: further investigate a better alternative
+ *
* Space optimized random access capable array of values with a fixed number of bits/value. Values
* are packed contiguously.
*
diff --git a/server/src/main/java/org/apache/lucene/util/packed/XPacked64SingleBlock.java b/server/src/main/java/org/apache/lucene/util/packed/XPacked64SingleBlock.java
index 0324522e9a68d..53cf4ed8e2273 100644
--- a/server/src/main/java/org/apache/lucene/util/packed/XPacked64SingleBlock.java
+++ b/server/src/main/java/org/apache/lucene/util/packed/XPacked64SingleBlock.java
@@ -25,8 +25,8 @@
/**
* Forked from Lucene 8.x; removed in Lucene 9.0
*
- * @todo further investigate a better alternative
- *
+ * TODO: further investigate a better alternative
+ *
* This class is similar to {@link Packed64} except that it trades space for speed by ensuring that
* a single block needs to be read/written in order to read/write a value.
*/
diff --git a/server/src/main/java/org/apache/lucene/util/packed/XPackedInts.java b/server/src/main/java/org/apache/lucene/util/packed/XPackedInts.java
index f94a4531a7db9..4260d34ead7c9 100644
--- a/server/src/main/java/org/apache/lucene/util/packed/XPackedInts.java
+++ b/server/src/main/java/org/apache/lucene/util/packed/XPackedInts.java
@@ -35,9 +35,9 @@
/**
* Forked from Lucene 8.x; removed in Lucene 8.9
- *
+ *
* Todo: further investigate a better alternative
- *
+ *
* Simplistic compression for array of unsigned long values. Each value is {@code >= 0} and {@code
* <=} a specified maximum value. The values are stored as packed ints, with each value consuming a
* fixed number of bits.
diff --git a/server/src/main/java/org/opensearch/action/AliasesRequest.java b/server/src/main/java/org/opensearch/action/AliasesRequest.java
index 4c5d5628b1aac..3632ba2d7304f 100644
--- a/server/src/main/java/org/opensearch/action/AliasesRequest.java
+++ b/server/src/main/java/org/opensearch/action/AliasesRequest.java
@@ -54,7 +54,7 @@ public interface AliasesRequest extends IndicesRequest.Replaceable {
/**
* Replaces current aliases with the provided aliases.
- *
+ *
* Sometimes aliases expressions need to be resolved to concrete aliases prior to executing the transport action.
*/
void replaceAliases(String... aliases);
diff --git a/server/src/main/java/org/opensearch/action/DocWriteRequest.java b/server/src/main/java/org/opensearch/action/DocWriteRequest.java
index df6414470ab6b..31f61e76c74ff 100644
--- a/server/src/main/java/org/opensearch/action/DocWriteRequest.java
+++ b/server/src/main/java/org/opensearch/action/DocWriteRequest.java
@@ -149,7 +149,7 @@ public interface DocWriteRequest extends IndicesRequest, Accountable {
/**
* If set, only perform this request if the document was last modification was assigned this primary term.
- *
+ *
* If the document last modification was assigned a different term a
* {@link org.opensearch.index.engine.VersionConflictEngineException} will be thrown.
*/
diff --git a/server/src/main/java/org/opensearch/action/DocWriteResponse.java b/server/src/main/java/org/opensearch/action/DocWriteResponse.java
index afdb1d3a0bdd9..e3052b3b80035 100644
--- a/server/src/main/java/org/opensearch/action/DocWriteResponse.java
+++ b/server/src/main/java/org/opensearch/action/DocWriteResponse.java
@@ -341,7 +341,7 @@ public XContentBuilder innerToXContent(XContentBuilder builder, Params params) t
/**
* Parse the output of the {@link #innerToXContent(XContentBuilder, Params)} method.
- *
+ *
* This method is intended to be called by subclasses and must be called multiple times to parse all the information concerning
* {@link DocWriteResponse} objects. It always parses the current token, updates the given parsing context accordingly
* if needed and then immediately returns.
diff --git a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java
index 0930bd2741810..5948dd3e2b7cb 100644
--- a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java
+++ b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java
@@ -50,7 +50,7 @@
/**
* Information about task operation failures
- *
+ *
* The class is final due to serialization limitations
*
* @opensearch.internal
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainRequest.java
index 625aa91e6ea7f..3dec781f0acf4 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainRequest.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainRequest.java
@@ -95,7 +95,7 @@ public ClusterAllocationExplainRequest(StreamInput in) throws IOException {
* Create a new allocation explain request. If {@code primary} is false, the first unassigned replica
* will be picked for explanation. If no replicas are unassigned, the first assigned replica will
* be explained.
- *
+ *
* Package private for testing.
*/
ClusterAllocationExplainRequest(String index, int shard, boolean primary, @Nullable String currentNode) {
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
index dd36b3b8db3ab..874713b51d627 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
@@ -46,6 +46,7 @@
import org.opensearch.core.xcontent.XContentBuilder;
import org.opensearch.discovery.DiscoveryStats;
import org.opensearch.http.HttpStats;
+import org.opensearch.index.SegmentReplicationRejectionStats;
import org.opensearch.index.stats.IndexingPressureStats;
import org.opensearch.index.stats.ShardIndexingPressureStats;
import org.opensearch.index.store.remote.filecache.FileCacheStats;
@@ -56,6 +57,8 @@
import org.opensearch.monitor.os.OsStats;
import org.opensearch.monitor.process.ProcessStats;
import org.opensearch.node.AdaptiveSelectionStats;
+import org.opensearch.node.NodesResourceUsageStats;
+import org.opensearch.repositories.RepositoriesStats;
import org.opensearch.script.ScriptCacheStats;
import org.opensearch.script.ScriptStats;
import org.opensearch.search.backpressure.stats.SearchBackpressureStats;
@@ -127,6 +130,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
@Nullable
private SearchBackpressureStats searchBackpressureStats;
+ @Nullable
+ private SegmentReplicationRejectionStats segmentReplicationRejectionStats;
+
@Nullable
private ClusterManagerThrottlingStats clusterManagerThrottlingStats;
@@ -142,6 +148,12 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
@Nullable
private SearchPipelineStats searchPipelineStats;
+ @Nullable
+ private NodesResourceUsageStats resourceUsageStats;
+
+ @Nullable
+ private RepositoriesStats repositoriesStats;
+
public NodeStats(StreamInput in) throws IOException {
super(in);
timestamp = in.readVLong();
@@ -198,6 +210,21 @@ public NodeStats(StreamInput in) throws IOException {
} else {
searchPipelineStats = null;
}
+ if (in.getVersion().onOrAfter(Version.V_2_12_0)) {
+ resourceUsageStats = in.readOptionalWriteable(NodesResourceUsageStats::new);
+ } else {
+ resourceUsageStats = null;
+ }
+ if (in.getVersion().onOrAfter(Version.V_2_12_0)) {
+ segmentReplicationRejectionStats = in.readOptionalWriteable(SegmentReplicationRejectionStats::new);
+ } else {
+ segmentReplicationRejectionStats = null;
+ }
+ if (in.getVersion().onOrAfter(Version.V_2_12_0)) {
+ repositoriesStats = in.readOptionalWriteable(RepositoriesStats::new);
+ } else {
+ repositoriesStats = null;
+ }
}
public NodeStats(
@@ -216,6 +243,7 @@ public NodeStats(
@Nullable DiscoveryStats discoveryStats,
@Nullable IngestStats ingestStats,
@Nullable AdaptiveSelectionStats adaptiveSelectionStats,
+ @Nullable NodesResourceUsageStats resourceUsageStats,
@Nullable ScriptCacheStats scriptCacheStats,
@Nullable IndexingPressureStats indexingPressureStats,
@Nullable ShardIndexingPressureStats shardIndexingPressureStats,
@@ -224,7 +252,9 @@ public NodeStats(
@Nullable WeightedRoutingStats weightedRoutingStats,
@Nullable FileCacheStats fileCacheStats,
@Nullable TaskCancellationStats taskCancellationStats,
- @Nullable SearchPipelineStats searchPipelineStats
+ @Nullable SearchPipelineStats searchPipelineStats,
+ @Nullable SegmentReplicationRejectionStats segmentReplicationRejectionStats,
+ @Nullable RepositoriesStats repositoriesStats
) {
super(node);
this.timestamp = timestamp;
@@ -241,6 +271,7 @@ public NodeStats(
this.discoveryStats = discoveryStats;
this.ingestStats = ingestStats;
this.adaptiveSelectionStats = adaptiveSelectionStats;
+ this.resourceUsageStats = resourceUsageStats;
this.scriptCacheStats = scriptCacheStats;
this.indexingPressureStats = indexingPressureStats;
this.shardIndexingPressureStats = shardIndexingPressureStats;
@@ -250,6 +281,8 @@ public NodeStats(
this.fileCacheStats = fileCacheStats;
this.taskCancellationStats = taskCancellationStats;
this.searchPipelineStats = searchPipelineStats;
+ this.segmentReplicationRejectionStats = segmentReplicationRejectionStats;
+ this.repositoriesStats = repositoriesStats;
}
public long getTimestamp() {
@@ -344,6 +377,11 @@ public AdaptiveSelectionStats getAdaptiveSelectionStats() {
return adaptiveSelectionStats;
}
+ @Nullable
+ public NodesResourceUsageStats getResourceUsageStats() {
+ return resourceUsageStats;
+ }
+
@Nullable
public ScriptCacheStats getScriptCacheStats() {
return scriptCacheStats;
@@ -387,6 +425,16 @@ public SearchPipelineStats getSearchPipelineStats() {
return searchPipelineStats;
}
+ @Nullable
+ public SegmentReplicationRejectionStats getSegmentReplicationRejectionStats() {
+ return segmentReplicationRejectionStats;
+ }
+
+ @Nullable
+ public RepositoriesStats getRepositoriesStats() {
+ return repositoriesStats;
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
@@ -430,6 +478,15 @@ public void writeTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(Version.V_2_9_0)) {
out.writeOptionalWriteable(searchPipelineStats);
}
+ if (out.getVersion().onOrAfter(Version.V_2_12_0)) {
+ out.writeOptionalWriteable(resourceUsageStats);
+ }
+ if (out.getVersion().onOrAfter(Version.V_2_12_0)) {
+ out.writeOptionalWriteable(segmentReplicationRejectionStats);
+ }
+ if (out.getVersion().onOrAfter(Version.V_2_12_0)) {
+ out.writeOptionalWriteable(repositoriesStats);
+ }
}
@Override
@@ -520,7 +577,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
if (getSearchPipelineStats() != null) {
getSearchPipelineStats().toXContent(builder, params);
}
+ if (getResourceUsageStats() != null) {
+ getResourceUsageStats().toXContent(builder, params);
+ }
+ if (getSegmentReplicationRejectionStats() != null) {
+ getSegmentReplicationRejectionStats().toXContent(builder, params);
+ }
+ if (getRepositoriesStats() != null) {
+ getRepositoriesStats().toXContent(builder, params);
+ }
return builder;
}
}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
index b0caa469033eb..fc72668d36413 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
@@ -213,7 +213,10 @@ public enum Metric {
WEIGHTED_ROUTING_STATS("weighted_routing"),
FILE_CACHE_STATS("file_cache"),
TASK_CANCELLATION("task_cancellation"),
- SEARCH_PIPELINE("search_pipeline");
+ SEARCH_PIPELINE("search_pipeline"),
+ RESOURCE_USAGE_STATS("resource_usage_stats"),
+ SEGMENT_REPLICATION_BACKPRESSURE("segment_replication_backpressure"),
+ REPOSITORIES("repositories");
private String metricName;
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
index 615abbaef845d..99cf42cfdc4d0 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
@@ -124,7 +124,10 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) {
NodesStatsRequest.Metric.WEIGHTED_ROUTING_STATS.containedIn(metrics),
NodesStatsRequest.Metric.FILE_CACHE_STATS.containedIn(metrics),
NodesStatsRequest.Metric.TASK_CANCELLATION.containedIn(metrics),
- NodesStatsRequest.Metric.SEARCH_PIPELINE.containedIn(metrics)
+ NodesStatsRequest.Metric.SEARCH_PIPELINE.containedIn(metrics),
+ NodesStatsRequest.Metric.RESOURCE_USAGE_STATS.containedIn(metrics),
+ NodesStatsRequest.Metric.SEGMENT_REPLICATION_BACKPRESSURE.containedIn(metrics),
+ NodesStatsRequest.Metric.REPOSITORIES.containedIn(metrics)
);
}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
index 8d82827f4ee50..e62c83490d810 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/TransportGetTaskAction.java
@@ -67,7 +67,7 @@
/**
* ActionType to get a single task. If the task isn't running then it'll try to request the status from request index.
- *
+ *
* The general flow is:
*