Snapshot_deletion threadpool active thread count stuck at 1 after encountering failure
Describe the bug
After upgrading form Opensearch 2.15.0 to 2.18.0, sometimes deleting a snapshot resulted in snapshot_deletion threadpool active thread count stuck at 1 after encountering failure. Logs on cluster_manager node showed:
[2025-05-13T23:53:31,217][INFO ][o.o.s.SnapshotsService ] [es-master-mis-30-2-0] deleting snapshots [smartsearch_min-2025-05-03.183003] from repository [s3_repository]
[2025-05-13T23:54:05,259][WARN ][o.o.r.b.BlobStoreRepository] [es-master-mis-30-2-0] [s3_repository] Failed to delete following blobs during snapshot delete : [Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][10], v2=__e05l2htCSi2L3NcdX5b2ww], Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][81], v2=__KNHrjCURQ0uCeIT35WyA_w], Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][56], v2=__yRISy4RpTL67SWuJVBbX4g]...
[2025-05-13T23:54:03,817][WARN ][o.o.r.b.BlobStoreRepository] [es-master-mis-30-2-0] [s3_repository] Failed to delete following blobs during snapshot delete : [Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][111], v2=__YW_-KjNvTzmcDBDL8Onm2Q], Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][147], v2=__-VaRQ2tTSEyD-8TLi2aFxw], Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][137], v2=__JBgcvFxWRH6vUN6pa_fXuw]...
[2025-05-13T23:54:04,347][WARN ][o.o.r.b.BlobStoreRepository] [es-master-mis-30-2-0] [s3_repository] Failed to delete following blobs during snapshot delete : [Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][24], v2=__AFiAbKSBR2Gq4XcaYqa-lA], Tuple [v1=[mis-30-2][indices][e7NMXjIlSOOS1MTWMTOr4Q][66], v2=__OHzTRuKpQPuarw2u3OtvGw]...
...
There were no pending tasks (checked via GET _cluster/pending_tasks). Thread count would become 0 only when the cluster manager node was restarted. Then when I tried to get the list of snapshots, the snapshot was gone, but trying to delete the next snapshot resulted in the same behavior with the "Failed to delete following blobs ..." log entry listing new blobs as well as blobs from previous snapshot deletion. These log entries only listed blobs that failed to be deleted but no cause of the failure, and there were no other log entries related to snapshot deletion error.
Also, I noticed whenever I made a repository setting update such changing chunk size, subsequent request to retrieve list of snapshots returned with the follow error:
{
"error": {
"root_cause": [
{
"type": "illegal_state_exception",
"reason": "Connection pool shut down"
}
],
"type": "illegal_state_exception",
"reason": "Connection pool shut down"
},
"status": 500
}
The request would work again only after restarting cluster manager nodes.
Related component
Storage:Snapshots
To Reproduce
- Create a snapshot management policy to take snapshot of the entire cluster every day at 12am and 12pm, and delete schedule of every day at 2am and 2pm. Keep minimum of 2 and max of 7 snapshots.
- Let policy take effect for a few days.
- Observe that snapshot deletion thread count remain at non-zero value.
Note that this behavior doesn't happen every time but once it does, subsequent snapshot deletions (after cluster manager node restart) will run into this behavior. Also, this symptom seems only to occur for our large clusters where there are daily indices with some indices have 250 shards and size of 3-4TB.
Expected behavior
Snapshot_deletion threadpool active thread count should become 0 even after encountering failure.
Additional Details
Plugins Please list all plugins currently enabled.
Screenshots If applicable, add screenshots to help explain your problem.
Host/Environment (please complete the following information):
- OS: [e.g. iOS]
- Version [e.g. 22]
Additional context Add any other context about the problem here.
Adding @shwetathareja @sachinpkale @Bukhtawar to please take a look and provide some insights. @tomchlee is there way this can be reproduced with some steps ?
Adding @shwetathareja @sachinpkale @Bukhtawar to please take a look and provide some insights. @tomchlee is there way this can be reproduced with some steps ?
@prudhvigodithi I have updated To Reproduce section in the Description.
@prudhvigodithi I have updated To Reproduce section in the Description.
Thanks @tomchlee, just curious I see you mentioned as 2.18, did you also try upgrading to 2.19.1 (the latest 2.x version) and have you seen this issue ? @getsaurabh02 @andrross
We haven't tried upgrading to 2.19.1 as this symptom has only manifested itself in our production clusters which have indices of that size. We could look into upgrading to 2.19.1 but it will take a bit of time to go through testing in lab.
@tomchlee this looks like an issue with stale shard deletion. It may require bouncing the OpenSearch process on data nodes as temporary mitigation in case it happen in your production. Can you please paste the full message why was it failing to delete the blob in this case?
cc: @ashking94 who worked on the snapshot code recently
@ashking94
We are catching all exceptions and then retrying, does it handle for the case where blobs might be deleted in S3 already but due to intermittent network error, request failed and then it would try again... https://github.com/opensearch-project/OpenSearch/blob/33ebf941607b088c700d74deb2088de79d646153/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java#L1667-L1678
Thanks for reporting the issue @tomchlee. I have couple of questions:
- Were you able to take a thread dump when the
snapshot_deletionactive thread count showed up as 1 continuously? Can you share the thread dump if you have one. I suspect if this could be some stats related issue. - Can you share the exact command for the update repository that led to list snapshot failing after that?
@tomchlee this looks like an issue with stale shard deletion. It may require bouncing the OpenSearch process on data nodes as temporary mitigation in case it happen in your production. Can you please paste the full message why was it failing to delete the blob in this case?
cc: @ashking94 who worked on the snapshot code recently
The log entry is quite long so I've truncated large chunk of middle portion which just list more tuples:
[2025-06-06T22:05:50,268][WARN ][o.o.r.b.BlobStoreRepository] [es-master-mis-30-3-2] [s3_repository] Failed to delete following blobs during snapshot delete : [Tuple [v1=[mis-30-3][indices][ONGyzZEZQ1WPGciAr_fkZQ][50], v2=__zjxXlrOaRGWgwe4z_b0Kyw], Tuple [v1=[mis-30-3][indices][RR4lDG0gRxuA13b7mgB1Wg][66], v2=__eI4ikUTCQlCsMckfEEyX_w], Tuple [v1=[mis-30-3][indices][o-X6U5HeSdeidBzJY2XAmQ][5], v2=__ZlQlkoO9TbS6qR7FdtpgRg], Tuple [v1=[mis-30-3][indices][2duN29lpRgSDUPwXK6NGNg][79], v2=__hx_wzhpnTwK4hO97S59UfA], Tuple [v1=[mis-30-3][indices][ljTDObqbTdGqTXdywDC0lg][107], v2=snap-4aLSK_YESZuCjAaOJy6XxQ.dat], Tuple [v1=[mis-30-3][indices][_GuNA3B9QGuKGkPYLcR1Cg][1], v2=__QeJNadMZTiWSIpUY4EsoYw], Tuple [v1=[mis-30-3][indices][sHHxP9GqQU-2hqbmBhOHdA][139], v2=___5ITtFpdR_OyzkYVs9434g], Tuple [v1=[mis-30-3][indices][gs0rbbAxRJuDBIck6BZfcQ][79], v2=__ECeywjzuShmYBdkSi8xAwA], Tuple [v1=[mis-30-3][indices][2duN29lpRgSDUPwXK6NGNg][102], v2=__le_gaXJLQ8a0Hj7BdrTRMA], Tuple [v1=[mis-30-3][indices][X8IDXnPuSxKIFkZpCRmYeA][128], v2=__voCnpHSHTOC0CkCFtSTUEw], Tuple [v1=[mis-30-3][indices][mwgniftDTa6WpZNy3glBTQ][68], v2=__GKvNcwmtSzOsKLDRjAl3UQ], Tuple [v1=[mis-30-3][indices][X8IDXnPuSxKIFkZpCRmYeA][108], v2=__WAhvgy3UTsyylLhljDvmpg], Tuple [v1=[mis-30-3][indices][iIED6ilrRQu_hqvLmx9o5g][137], v2=snap-amXHXu-ARoeVCtk3fr1cmg.dat], Tuple [v1=[mis-30-3][indices][_uukV0MESyyd9geXut9uMw][108], v2=__FG_HhGupRwm517zDcvgLfg], ... *** TRUNCATED *** ... Tuple [v1=[mis-30-3][indices][mwgniftDTa6WpZNy3glBTQ][46], v2=__1xM-NwbBSS21BW6mzFShOw], Tuple [v1=[mis-30-3][indices][iIED6ilrRQu_hqvLmx9o5g][81], v2=__GV7GUYo3QWWB42Mmf8yhCw], Tuple [v1=[mis-30-3][indices][ljTDObqbTdGqTXdywDC0lg][142], v2=index-jZRPP0E3Ti6VymwRK8ynig], Tuple [v1=[mis-30-3][indices][a13te36rQP-cisW8hGQDJw][135], v2=__3aUaGvNlTAOou6qiwDmsKg], Tuple [v1=[mis-30-3][indices][RR4lDG0gRxuA13b7mgB1Wg][93], v2=__-pZFhACTS-eIuNAy4j0_SQ], Tuple [v1=[mis-30-3][indices][qQQMvwLHSTGrJFVDszzuOQ][115], v2=__cm2txdrYT46KNI4VT82Wcg], Tuple [v1=[mis-30-3][indices][a13te36rQP-cisW8hGQDJw][26], v2=__KF95t3w1SIaVL-IKUWZpqg], Tuple [v1=[mis-30-3][indices][Mm_yUwerT1qJSR___QPUZw][0], v2=snap-8StXo3tsQ_qFbFU_ToHKaA.dat], Tuple [v1=[mis-30-3][indices][UOaIOyopTQ2M492ty3dc4g][95], v2=__cpChtqeXQgigNad66RwnAg], Tuple [v1=[mis-30-3][indices][X8IDXnPuSxKIFkZpCRmYeA][46], v2=__B0SV-8eXSHGIizmrps_7dA], Tuple [v1=[mis-30-3][indices][2duN29lpRgSDUPwXK6NGNg][23], v2=__KqwYvL3DS9mU4u7meOW5Eg]]
I didn't find any exception logged.
@tomchlee If possible, I think it would be super helpful if you could get a thread dump when one of these threads is stuck in the active state (jstack is a good option if you haven't done this before). This should give the stack trace for the thread that is running.
@tomchlee If possible, I think it would be super helpful if you could get a thread dump when one of these threads is stuck in the active state (jstack is a good option if you haven't done this before). This should give the stack trace for the thread that is running.
Please find threaddump below (snapshot deletion thread count was stuck at 4 on this node):
Thanks for reporting the issue @tomchlee. I have couple of questions:
2. Can you share the exact command for the update repository that led to list snapshot failing after that?
Request:
PUT _snapshot/s3_repository
{
"type": "s3",
"settings": {
"bucket": "mis-dev-es-snapshots-use1",
"base_path": "mis-30",
"chunk_size": "100mb",
"server_side_encryption": "true",
"max_restore_bytes_per_sec": "100mb",
"compress": "true",
"max_snapshot_bytes_per_sec": "100mb"
}
}
Response:
{
"error": {
"root_cause": [
{
"type": "repository_verification_exception",
"reason": "[s3_repository] path [mis-30] is not accessible on cluster-manager node"
}
],
"type": "repository_verification_exception",
"reason": "[s3_repository] path [mis-30] is not accessible on cluster[-](https://cerebro.prestage.mis.lab.ppops.net/#)manager node",
"caused_by": { -
"type": "illegal_state_exception",
"reason": "Connection pool shut down"
}
},
"status": 500
}
However, subsequent request to retrieve s3 repo settings was successful with updated setting values but request to list snapshots resulted in response below:
{
"error": {
"root_cause": [
{
"type": "illegal_state_exception",
"reason": "Connection pool shut down"
}
],
"type": "illegal_state_exception",
"reason": "Connection pool shut down"
},
"status": 500
}
Note that this behavior also happens on clusters that do not exhibit stuck snapshot deletion thread count issue.
After upgrading our lab clusters to 2.19.1, "Connection pool shut down" issue no longer occurs.
We have upgraded production clusters that were exhibiting snapshot deletion thread count stuck at non-zero to version 2.19.1 but the stuck thread count behavior is still occurring.
The thread dump does indeed show threads blocked on a BlobStoreRepository.deleteContainer call:
"opensearch[es-master-mis-30-3-0][snapshot_deletion][T#32]" #141 [256] daemon prio=5 os_prio=0 cpu=2031.84ms elapsed=6755.91s allocated=479M defined_classes=4 tid=0x00007f8902692720 nid=256 waiting on condition [0x00007f88ffb1a000]
java.lang.Thread.State: WAITING (parking)
at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
- parking to wait for <0x000000068949b8d0> (a org.opensearch.common.util.concurrent.BaseFuture$Sync)
at java.util.concurrent.locks.LockSupport.park([email protected]/Unknown Source)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/Unknown Source)
at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly([email protected]/Unknown Source)
at org.opensearch.common.util.concurrent.BaseFuture$Sync.get(BaseFuture.java:272)
at org.opensearch.common.util.concurrent.BaseFuture.get(BaseFuture.java:104)
at org.opensearch.common.util.concurrent.FutureUtils.get(FutureUtils.java:74)
at org.opensearch.action.support.AdapterActionFuture.actionGet(AdapterActionFuture.java:55)
at org.opensearch.repositories.blobstore.BlobStoreRepository.deleteContainer(BlobStoreRepository.java:2225)
at org.opensearch.repositories.blobstore.BlobStoreRepository.lambda$executeOneStaleIndexDelete$44(BlobStoreRepository.java:2185)
at org.opensearch.repositories.blobstore.BlobStoreRepository$$Lambda/0x0000000800f85c40.get(Unknown Source)
at org.opensearch.action.ActionRunnable.lambda$supply$0(ActionRunnable.java:74)
at org.opensearch.action.ActionRunnable$$Lambda/0x00000008009bab18.accept(Unknown Source)
at org.opensearch.action.ActionRunnable$2.doRun(ActionRunnable.java:89)
at org.opensearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:1005)
at org.opensearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:52)
at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/Unknown Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/Unknown Source)
at java.lang.Thread.runWith([email protected]/Unknown Source)
at java.lang.Thread.run([email protected]/Unknown Source)
@ashking94 Have you had a chance to look at this further?
Thanks @tomchlee for the details. Are there any other log lines that you see around the issue (with the snapshot shard blobs deletion) that is happening when the threads get stuck?
@andrross taking a look now.
@ashking94 that is the only type of log lines i see that were snapshot related. there was nothing about what caused the failed to delete.
@tomchlee - For cluster which are on 2.18 or earlier versions supporting async deletion, you can disable async deletion by setting cluster.snapshot.async-deletion.enable to false.
@ashking94 that is the only type of log lines i see that were snapshot related. there was nothing about what caused the failed to delete.
Are there any other exception that you see in the logs around the same time when the active snapshot deletion thread pops up?
Based on the analysis of the code and diagnostic data shared, I can think of the following cases where the snapshot_deletion thread can wait indefinitely for future to complete.
1. Race condition in Completion Handler
S3BlobContainer -> deleteAsync:
@Override
public void onComplete() {
if (!objectsToDelete.isEmpty()) {
deletionChain = S3AsyncDeleteHelper.executeDeleteChain(
s3AsyncClient,
blobStore,
objectsToDelete,
deletionChain,
null
);
}
deletionChain.whenComplete((v, throwable) -> { // Race condition point
if (throwable != null) {
listingFuture.completeExceptionally(throwable);
} else {
listingFuture.complete(null);
}
});
}
Problem: If deletionChain completes before whenComplete is attached:
- The completion handler is never executed
- listingFuture never completes
- The deletion thread remains stuck waiting for completion
- No timeout mechanism exists to break this deadlock
2. Never Completing CompletableFuture
// In S3AsyncDeleteHelper.java
public static CompletableFuture<Void> executeDeleteChain(...) {
return currentChain.thenCompose(v -> executeDeleteBatches(...));
}
static CompletableFuture<Void> executeDeleteBatches(...) {
CompletableFuture<Void> allDeletesFuture = CompletableFuture.completedFuture(null);
for (List<String> batch : batches) {
allDeletesFuture = allDeletesFuture.thenCompose(
v -> executeSingleDeleteBatch(...) // This future might never complete
);
}
return allDeletesFuture;
}
static CompletableFuture<Void> executeSingleDeleteBatch(...) {
return s3AsyncClient.deleteObjects(deleteRequest) // S3 call might hang
.thenApply(S3AsyncDeleteHelper::processDeleteResponse);
}
Problem: If any delete operation hangs:
- The CompletableFuture from s3AsyncClient.deleteObjects() never completes
- The chain of thenCompose operations stalls
- No timeout is enforced on the S3 client operation
- The entire deletion chain remains incomplete
- Thread waiting on deletionChain.whenComplete() remains stuck
Call hierarchy that leads to the stuck thread:
BlobStoreRepository.deleteAsync()
└─> Subscriber.onComplete()
└─> S3AsyncDeleteHelper.executeDeleteChain()
└─> executeDeleteBatches()
└─> executeSingleDeleteBatch()
└─> s3AsyncClient.deleteObjects() // Hangs here
└─> CompletableFuture never completes
└─> deletionChain.whenComplete() never executes
└─> listingFuture never completes
└─> Thread remains stuck
Both scenarios result in the same thread stack trace:
"opensearch[...][snapshot_deletion][T#32]" ... waiting on condition
at jdk.internal.misc.Unsafe.park(Native Method)
at java.util.concurrent.locks.LockSupport.park
at AbstractQueuedSynchronizer.acquire
at AbstractQueuedSynchronizer.acquireSharedInterruptibly
at BaseFuture$Sync.get(BaseFuture.java:272)
at BaseFuture.get(BaseFuture.java:104)
at BlobStoreRepository.deleteContainer(BlobStoreRepository.java:2225)
The key difference is:
In scenario 1, the CompletableFuture completes but its completion handler is missed In scenario 2, the CompletableFuture never completes, so the thread waits indefinitely
As next steps, let me make code changes to get this addressed in next 2.19 minor release 2.19.3. Apart from these, I will also make some changes to make the code slightly defensive to ensure that we don't run into this issue.
cc @tomchlee
Tried validating the 1st theory through a opensearch UT and it did not work. Also tried this below snippet of code and this disproves the 1st theory.
CompletableFuture<String> future = CompletableFuture.completedFuture("Hello");
System.out.println("Before whenComplete");
future.whenComplete((result, error) -> {
System.out.println("Inside whenComplete: " + result);
});
System.out.println("After whenComplete");
OpenSearch UT (can be run in S3BlobStoreContainerTests) that I tested:
public void testDeleteAsyncCompletionRaceCondition() throws Exception {
final String bucketName = randomAlphaOfLengthBetween(1, 10);
final BlobPath blobPath = new BlobPath();
final S3BlobStore blobStore = mock(S3BlobStore.class);
when(blobStore.bucket()).thenReturn(bucketName);
when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher());
when(blobStore.getBulkDeletesSize()).thenReturn(1);
final S3AsyncClient s3AsyncClient = mock(S3AsyncClient.class);
final AmazonAsyncS3Reference asyncClientReference = mock(AmazonAsyncS3Reference.class);
when(blobStore.asyncClientReference()).thenReturn(asyncClientReference);
AmazonAsyncS3WithCredentials amazonAsyncS3WithCredentials = AmazonAsyncS3WithCredentials.create(
s3AsyncClient,
s3AsyncClient,
s3AsyncClient,
null
);
when(asyncClientReference.get()).thenReturn(amazonAsyncS3WithCredentials);
// Control the completion of delete operation
CompletableFuture<DeleteObjectsResponse> deleteFuture = new CompletableFuture<>();
when(s3AsyncClient.deleteObjects(any(DeleteObjectsRequest.class)))
.thenReturn(deleteFuture);
final ListObjectsV2Publisher listPublisher = mock(ListObjectsV2Publisher.class);
final AtomicBoolean hasEmittedItem = new AtomicBoolean(false);
final CountDownLatch onNextCalled = new CountDownLatch(1);
final CountDownLatch waitForDeleteToComplete = new CountDownLatch(1);
doAnswer(invocation -> {
Subscriber<? super ListObjectsV2Response> subscriber = invocation.getArgument(0);
subscriber.onSubscribe(new Subscription() {
@Override
public void request(long n) {
if (!hasEmittedItem.getAndSet(true)) {
// Emit one item
subscriber.onNext(
ListObjectsV2Response.builder()
.contents(S3Object.builder().key("test-key").size(100L).build())
.build()
);
onNextCalled.countDown();
try {
// Wait for delete operation to complete
waitForDeleteToComplete.await(1, TimeUnit.SECONDS);
deleteFuture.complete(DeleteObjectsResponse.builder().build());
// Complete after delete is done
subscriber.onComplete();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
}
@Override
public void cancel() {}
});
return null;
}).when(listPublisher).subscribe(ArgumentMatchers.<Subscriber<ListObjectsV2Response>>any());
when(s3AsyncClient.listObjectsV2Paginator(any(ListObjectsV2Request.class)))
.thenReturn(listPublisher);
final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore);
CountDownLatch listenerLatch = new CountDownLatch(1);
AtomicBoolean listenerCalled = new AtomicBoolean(false);
blobContainer.deleteAsync(new ActionListener<>() {
@Override
public void onResponse(DeleteResult deleteResult) {
listenerCalled.set(true);
listenerLatch.countDown();
}
@Override
public void onFailure(Exception e) {
listenerCalled.set(true);
listenerLatch.countDown();
}
});
// Wait for onNext to be called
assertTrue("onNext should have been called", onNextCalled.await(1, TimeUnit.SECONDS));
// Complete the delete operation
// deleteFuture.complete(DeleteObjectsResponse.builder().build());
waitForDeleteToComplete.countDown();
// Check if listener was called
assertFalse("Listener should not have been called", listenerLatch.await(5000, TimeUnit.MILLISECONDS));
assertFalse("Listener should not have been called", listenerCalled.get());
// Verify the sequence
verify(s3AsyncClient, times(1)).listObjectsV2Paginator(any(ListObjectsV2Request.class));
verify(s3AsyncClient, times(1)).deleteObjects(any(DeleteObjectsRequest.class));
}
@ashking94 that is the only type of log lines i see that were snapshot related. there was nothing about what caused the failed to delete.
Are there any other exception that you see in the logs around the same time when the active snapshot deletion thread pops up?
No, I didn't find any other exception.
@tomchlee - For cluster which are on 2.18 or earlier versions supporting async deletion, you can disable async deletion by setting
cluster.snapshot.async-deletion.enableto false.
We've upgraded to 2.19.1. Does the setting still apply? Will snapshot management policy still work if setting is set to false?
We've upgraded to 2.19.1. Does the setting still apply? Will snapshot management policy still work if setting is set to false?
The setting has been removed with 2.19 version which means that async deletion is the default and the only option.
@andrross would you be able to review https://github.com/opensearch-project/OpenSearch/pull/18583/? This should fix the issue here.
Hi @tomchlee, are you still seeing the issue? If so, can you tell share the debug logs?
Hi @tomchlee, are you still seeing the issue? If so, can you tell share the debug logs?
@ashking94 we're still seeing same issue on both 2.18.0 and 2.19.1 with same log entries. we still don't see a release with the fix in https://github.com/opensearch-project/OpenSearch/pull/18583
@tomchlee Have you tried with latest 2.19.3 release? https://opensearch.org/artifacts/by-version/#release-2-19-3
@tomchlee Have you tried with latest 2.19.3 release? https://opensearch.org/artifacts/by-version/#release-2-19-3
@rishabh6788 no i haven't. 2.19.3 release notes didn't include https://github.com/opensearch-project/OpenSearch/issues/url
@tomchlee can you try 2.19.3 and let me know if you still see the issue. Even if you dont see the issue, do share the debug logs. I have added the required debug logs to really understand on on the overall behaviour along with the fix for the issue that you have seen.
@tomchlee can you try 2.19.3 and let me know if you still see the issue. Even if you dont see the issue, do share the debug logs. I have added the required debug logs to really understand on on the overall behaviour along with the fix for the issue that you have seen.
@ashking94 what were changes in 2.19.3 that could've addressed this issue? we only see this behavior in our prod deployment and we already tried upgrading to 2.19.1 so i don't want to update prod environment without a good reason.