From 7ee65b15119432a9d2f01ae9c52ccbb04a37471f Mon Sep 17 00:00:00 2001 From: RS146BIJAY Date: Tue, 2 Dec 2025 11:50:53 +0530 Subject: [PATCH] Fix indexing regression and bug fixes for grouping criteria Signed-off-by: RS146BIJAY --- CHANGELOG.md | 2 + .../action/bulk/TransportShardBulkAction.java | 15 +- .../org/opensearch/index/IndexSettings.java | 4 +- .../index/engine/CompositeIndexWriter.java | 126 ++++++--------- .../index/mapper/MapperService.java | 5 +- .../bulk/TransportShardBulkActionTests.java | 144 ------------------ .../CompositeIndexWriterForAppendTests.java | 45 +++++- ...riaBasedCompositeIndexWriterBaseTests.java | 1 + 8 files changed, 101 insertions(+), 241 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4433e1bbfe9fb..e199cdb55108d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -100,6 +100,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fixed handling of property index in BulkRequest during deserialization ([#20132](https://github.com/opensearch-project/OpenSearch/pull/20132)) - Fix negative CPU usage values in node stats ([#19120](https://github.com/opensearch-project/OpenSearch/issues/19120)) - Fix duplicate registration of FieldDataCache dynamic setting ([20140](https://github.com/opensearch-project/OpenSearch/pull/20140)) +- Fix toBuilder method in EngineConfig to include mergedSegmentTransferTracker([20105](https://github.com/opensearch-project/OpenSearch/pull/20105)) +- Fix indexing regression and bug fixes for grouping criteria. ([20145](https://github.com/opensearch-project/OpenSearch/pull/20145)) ### Dependencies - Bump Apache Lucene from 10.3.1 to 10.3.2 ([#20026](https://github.com/opensearch-project/OpenSearch/pull/20026)) diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java index 37ca3161117d5..e9e9cb7f37532 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java @@ -87,7 +87,6 @@ import org.opensearch.index.IndexingPressureService; import org.opensearch.index.SegmentReplicationPressureService; import org.opensearch.index.engine.Engine; -import org.opensearch.index.engine.LookupMapLockAcquisitionException; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.index.get.GetResult; import org.opensearch.index.mapper.MapperException; @@ -728,15 +727,7 @@ && isConflictException(executionResult.getFailure().getCause()) && context.getRetryCounter() < ((UpdateRequest) docWriteRequest).retryOnConflict()) { context.resetForExecutionForRetry(); return; - } else if (isFailed - && context.getPrimary() != null - && context.getPrimary().indexSettings() != null - && context.getPrimary().indexSettings().isContextAwareEnabled() - && isLookupMapLockAcquisitionException(executionResult.getFailure().getCause()) - && context.getRetryCounter() < context.getPrimary().indexSettings().getMaxRetryOnLookupMapAcquisitionException()) { - context.resetForExecutionForRetry(); - return; - } + } final BulkItemResponse response; if (isUpdate) { response = processUpdateResponse((UpdateRequest) docWriteRequest, context.getConcreteIndex(), executionResult, updateResult); @@ -765,10 +756,6 @@ private static boolean isConflictException(final Exception e) { return ExceptionsHelper.unwrapCause(e) instanceof VersionConflictEngineException; } - private static boolean isLookupMapLockAcquisitionException(final Exception e) { - return ExceptionsHelper.unwrapCause(e) instanceof LookupMapLockAcquisitionException; - } - /** * Creates a new bulk item result from the given requests and result of performing the update operation on the shard. */ diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 7c5be554a7760..7e808205250c7 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -498,9 +498,9 @@ public static IndexMergePolicy fromString(String text) { */ public static final Setting INDEX_MAX_RETRY_ON_LOOKUP_MAP_LOCK_ACQUISITION_EXCEPTION = Setting.intSetting( "index.context_aware.max_retry_on_lookup_map_acquisition_exception", - 15, - 5, 100, + 5, + 500, Setting.Property.IndexScope, Property.Dynamic ); diff --git a/server/src/main/java/org/opensearch/index/engine/CompositeIndexWriter.java b/server/src/main/java/org/opensearch/index/engine/CompositeIndexWriter.java index e35c5dd1145d4..b4c89e6e08377 100644 --- a/server/src/main/java/org/opensearch/index/engine/CompositeIndexWriter.java +++ b/server/src/main/java/org/opensearch/index/engine/CompositeIndexWriter.java @@ -40,7 +40,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -48,7 +47,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.stream.Collectors; import static org.opensearch.index.BucketedCompositeDirectory.CHILD_DIRECTORY_PREFIX; @@ -209,12 +207,12 @@ public CriteriaBasedIndexWriterLookup getLookupMap() { * each refresh cycle. * */ - public static final class CriteriaBasedIndexWriterLookup implements Closeable { + public static class CriteriaBasedIndexWriterLookup implements Closeable { private final Map criteriaBasedIndexWriterMap; private final Map lastDeleteEntrySet; private final Map criteria; private final ReentrantReadWriteLock mapLock; - private final CriteriaBasedWriterLock mapReadLock; + CriteriaBasedWriterLock mapReadLock; private final ReleasableLock mapWriteLock; private final long version; private boolean closed; @@ -300,7 +298,7 @@ public boolean isClosed() { return closed; } - private static final class CriteriaBasedWriterLock implements Releasable { + static class CriteriaBasedWriterLock implements Releasable { private final Lock lock; // a per-thread count indicating how many times the thread has entered the lock; only works if assertions are enabled private final ThreadLocal holdingThreads; @@ -405,9 +403,9 @@ public Term getTerm() { * * @opensearch.internal */ - final static class LiveIndexWriterDeletesMap { + static class LiveIndexWriterDeletesMap { // All writes (adds and deletes) go into here: - final CriteriaBasedIndexWriterLookup current; + CriteriaBasedIndexWriterLookup current; // Used while refresh is running, and to hold adds/deletes until refresh finishes. We read from both current and old on lookup: final CriteriaBasedIndexWriterLookup old; @@ -468,16 +466,31 @@ String getCriteriaForDoc(BytesRef key) { DisposableIndexWriter computeIndexWriterIfAbsentForCriteria( String criteria, CheckedBiFunction indexWriterSupplier, - ShardId shardId + ShardId shardId, + int maxRetryOnLookupMapAcquisitionException ) { boolean success = false; CriteriaBasedIndexWriterLookup current = null; try { - current = getCurrentMap(); + int counter = 0; + while ((current == null || current.isClosed()) && counter < maxRetryOnLookupMapAcquisitionException) { + // This function acquires a first read lock on a map which does not have any write lock present. Current keeps + // on getting rotated during refresh, so there will be one current on which read lock can be obtained. + // Validate that no write lock is applied on the map and the map is not closed. Idea here is write lock was + // never applied on this map as write lock gets only during closing time. We are doing this instead of acquire, + // because acquire can also apply a read lock in case refresh completed and map is closed. + current = this.current.mapReadLock.tryAcquire(); + if (current != null && current.isClosed() == true) { + current.mapReadLock.close(); + current = null; + } + + ++counter; + } + if (current == null || current.isClosed()) { throw new LookupMapLockAcquisitionException(shardId, "Unable to obtain lock on the current Lookup map", null); } - DisposableIndexWriter writer = current.computeIndexWriterIfAbsentForCriteria(criteria, indexWriterSupplier); success = true; return writer; @@ -489,15 +502,6 @@ DisposableIndexWriter computeIndexWriterIfAbsentForCriteria( } } - // This function acquires a first read lock on a map which does not have any write lock present. Current keeps - // on getting rotated during refresh, so there will be one current on which read lock can be obtained. - // Validate that no write lock is applied on the map and the map is not closed. Idea here is write lock was - // never applied on this map as write lock gets only during closing time. We are doing this instead of acquire, - // because acquire can also apply a read lock in case refresh completed and map is closed. - CriteriaBasedIndexWriterLookup getCurrentMap() { - return current.mapReadLock.tryAcquire(); - } - // Used for Test Case. ReleasableLock acquireCurrentWriteLock() { return current.mapWriteLock.acquire(); @@ -672,7 +676,8 @@ DisposableIndexWriter computeIndexWriterIfAbsentForCriteria( return currentLiveIndexWriterDeletesMap.computeIndexWriterIfAbsentForCriteria( criteria, indexWriterSupplier, - engineConfig.getShardId() + engineConfig.getShardId(), + engineConfig.getIndexSettings().getMaxRetryOnLookupMapAcquisitionException() ); } @@ -684,12 +689,8 @@ public Map getMarkForRefreshIndexWriterMap() { public long getFlushingBytes() { ensureOpen(); long flushingBytes = 0; - Collection currentWriterSet = liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - for (IndexWriter currentWriter : currentWriterSet) { - flushingBytes += currentWriter.getFlushingBytes(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values()) { + flushingBytes += disposableIndexWriter.getIndexWriter().getFlushingBytes(); } return flushingBytes + accumulatingIndexWriter.getFlushingBytes(); @@ -699,13 +700,8 @@ public long getFlushingBytes() { public long getPendingNumDocs() { ensureOpen(); long pendingNumDocs = 0; - Collection currentWriterSet = liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - ; - for (IndexWriter currentWriter : currentWriterSet) { - pendingNumDocs += currentWriter.getPendingNumDocs(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values()) { + pendingNumDocs += disposableIndexWriter.getIndexWriter().getPendingNumDocs(); } // TODO: Should we add docs for old writer as well? @@ -733,24 +729,17 @@ public boolean hasUncommittedChanges() { @Override public Throwable getTragicException() { - Collection currentWriterSet = liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - for (IndexWriter writer : currentWriterSet) { - if (writer.isOpen() == false && writer.getTragicException() != null) { - return writer.getTragicException(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == false + && disposableIndexWriter.getIndexWriter().getTragicException() != null) { + return disposableIndexWriter.getIndexWriter().getTragicException(); } } - Collection oldWriterSet = liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - ; - for (IndexWriter writer : oldWriterSet) { - if (writer.isOpen() == false && writer.getTragicException() != null) { - return writer.getTragicException(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == false + && disposableIndexWriter.getIndexWriter().getTragicException() != null) { + return disposableIndexWriter.getIndexWriter().getTragicException(); } } @@ -765,27 +754,19 @@ public Throwable getTragicException() { public final long ramBytesUsed() { ensureOpen(); long ramBytesUsed = 0; - Collection currentWriterSet = liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); try (ReleasableLock ignore = liveIndexWriterDeletesMap.current.mapWriteLock.acquire()) { - for (IndexWriter indexWriter : currentWriterSet) { - if (indexWriter.isOpen() == true) { - ramBytesUsed += indexWriter.ramBytesUsed(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == true) { + ramBytesUsed += disposableIndexWriter.getIndexWriter().ramBytesUsed(); } } } - Collection oldWriterSet = liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); try (ReleasableLock ignore = liveIndexWriterDeletesMap.old.mapWriteLock.acquire()) { - for (IndexWriter indexWriter : oldWriterSet) { - if (indexWriter.isOpen() == true) { - ramBytesUsed += indexWriter.ramBytesUsed(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == true) { + ramBytesUsed += disposableIndexWriter.getIndexWriter().ramBytesUsed(); } } } @@ -813,24 +794,15 @@ public final synchronized Iterable> getLiveCommitData( public void rollback() throws IOException { if (shouldClose()) { - Collection currentWriterSet = liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - - for (IndexWriter indexWriter : currentWriterSet) { - if (indexWriter.isOpen() == true) { - indexWriter.rollback(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.current.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == true) { + disposableIndexWriter.getIndexWriter().rollback(); } } - Collection oldWriterSet = liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values() - .stream() - .map(DisposableIndexWriter::getIndexWriter) - .collect(Collectors.toSet()); - for (IndexWriter indexWriter : oldWriterSet) { - if (indexWriter.isOpen() == true) { - indexWriter.rollback(); + for (DisposableIndexWriter disposableIndexWriter : liveIndexWriterDeletesMap.old.criteriaBasedIndexWriterMap.values()) { + if (disposableIndexWriter.getIndexWriter().isOpen() == true) { + disposableIndexWriter.getIndexWriter().rollback(); } } diff --git a/server/src/main/java/org/opensearch/index/mapper/MapperService.java b/server/src/main/java/org/opensearch/index/mapper/MapperService.java index b0acdceeff9ce..0b3f8516fc5c2 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/opensearch/index/mapper/MapperService.java @@ -84,6 +84,7 @@ import java.util.function.BooleanSupplier; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; @@ -690,7 +691,9 @@ public boolean isCompositeIndexPresent() { } public Set getCompositeFieldTypes() { - return compositeMappedFieldTypes; + return compositeMappedFieldTypes.stream() + .filter(compositeMappedFieldType -> compositeMappedFieldType instanceof StarTreeMapper.StarTreeFieldType) + .collect(Collectors.toSet()); } private Set getCompositeFieldTypesFromMapper() { diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java index 6340cfe145272..00bd99dd4b349 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java @@ -74,7 +74,6 @@ import org.opensearch.index.SegmentReplicationPressureService; import org.opensearch.index.VersionType; import org.opensearch.index.engine.Engine; -import org.opensearch.index.engine.LookupMapLockAcquisitionException; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.Mapping; @@ -109,8 +108,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.LongSupplier; -import static org.opensearch.common.util.FeatureFlags.CONTEXT_AWARE_MIGRATION_EXPERIMENTAL_FLAG; -import static org.opensearch.index.IndexSettingsTests.newIndexMeta; import static org.opensearch.index.remote.RemoteStoreTestsHelper.createIndexSettings; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; @@ -1221,72 +1218,6 @@ public void testRetries() throws Exception { latch.await(); } - @LockFeatureFlag(CONTEXT_AWARE_MIGRATION_EXPERIMENTAL_FLAG) - public void testRetriesWithLookupMapLockAcquisitionException() throws Exception { - Settings settings = Settings.builder().put(IndexSettings.INDEX_CONTEXT_AWARE_ENABLED_SETTING.getKey(), true).build(); - IndexSettings indexSettings = new IndexSettings(newIndexMeta("test", settings), Settings.EMPTY); - UpdateRequest writeRequest = new UpdateRequest("index", "id").doc(Requests.INDEX_CONTENT_TYPE, "field", "value"); - BulkItemRequest primaryRequest = new BulkItemRequest(0, writeRequest); - - IndexRequest updateResponse = new IndexRequest("index").id("id").source(Requests.INDEX_CONTENT_TYPE, "field", "value"); - - Exception err = new LookupMapLockAcquisitionException(shardId, "Unable to obtain lock on the current Lookup map", null); - Engine.IndexResult lookupMapExceptionResult = new Engine.IndexResult(err, 0); - Engine.IndexResult mappingUpdate = new Engine.IndexResult( - new Mapping(null, mock(RootObjectMapper.class), new MetadataFieldMapper[0], Collections.emptyMap()) - ); - Translog.Location resultLocation = new Translog.Location(42, 42, 42); - Engine.IndexResult success = new FakeIndexResult(1, 1, 13, true, resultLocation); - - IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenAnswer(ir -> { - if (randomBoolean()) { - return lookupMapExceptionResult; - } else { - return success; - } - }); - when(shard.indexSettings()).thenReturn(indexSettings); - when(shard.shardId()).thenReturn(shardId); - when(shard.mapperService()).thenReturn(mock(MapperService.class)); - - UpdateHelper updateHelper = mock(UpdateHelper.class); - when(updateHelper.prepare(any(), eq(shard), any())).thenReturn( - new UpdateHelper.Result( - updateResponse, - randomBoolean() ? DocWriteResponse.Result.CREATED : DocWriteResponse.Result.UPDATED, - Collections.singletonMap("field", "value"), - Requests.INDEX_CONTENT_TYPE - ) - ); - - BulkItemRequest[] items = new BulkItemRequest[] { primaryRequest }; - BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, RefreshPolicy.NONE, items); - - final CountDownLatch latch = new CountDownLatch(1); - TransportShardBulkAction.performOnPrimary( - bulkShardRequest, - shard, - updateHelper, - threadPool::absoluteTimeInMillis, - new NoopMappingUpdatePerformer(), - listener -> listener.onResponse(null), - new LatchedActionListener<>(ActionTestUtils.assertNoFailureListener(result -> { - assertThat(((WritePrimaryResult) result).location, equalTo(resultLocation)); - BulkItemResponse primaryResponse = result.replicaRequest().items()[0].getPrimaryResponse(); - assertThat(primaryResponse.getItemId(), equalTo(0)); - assertThat(primaryResponse.getId(), equalTo("id")); - assertThat(primaryResponse.getOpType(), equalTo(DocWriteRequest.OpType.UPDATE)); - DocWriteResponse response = primaryResponse.getResponse(); - assertThat(response.status(), equalTo(RestStatus.CREATED)); - assertThat(response.getSeqNo(), equalTo(13L)); - }), latch), - threadPool, - Names.WRITE - ); - latch.await(); - } - public void testUpdateWithRetryOnConflict() throws IOException, InterruptedException { IndexSettings indexSettings = new IndexSettings(indexMetadata(), Settings.EMPTY); @@ -1363,81 +1294,6 @@ public void testUpdateWithRetryOnConflict() throws IOException, InterruptedExcep }); } - @LockFeatureFlag(CONTEXT_AWARE_MIGRATION_EXPERIMENTAL_FLAG) - public void testRetriesWithLookupMapLockAcquisitionExceptionWithMaxRetry() throws IOException, InterruptedException { - int retryCount = randomIntBetween(6, 10); - Settings settings = Settings.builder() - .put(IndexSettings.INDEX_CONTEXT_AWARE_ENABLED_SETTING.getKey(), true) - .put(IndexSettings.INDEX_MAX_RETRY_ON_LOOKUP_MAP_LOCK_ACQUISITION_EXCEPTION.getKey(), retryCount) - .build(); - IndexSettings indexSettings = new IndexSettings(newIndexMeta("test", settings), Settings.EMPTY); - - int nItems = randomIntBetween(2, 5); - List items = new ArrayList<>(nItems); - for (int i = 0; i < nItems; i++) { - UpdateRequest updateRequest = new UpdateRequest("index", "id").doc(Requests.INDEX_CONTENT_TYPE, "field", "value"); - items.add(new BulkItemRequest(i, updateRequest)); - } - - IndexRequest updateResponse = new IndexRequest("index").id("id").source(Requests.INDEX_CONTENT_TYPE, "field", "value"); - - Exception err = new LookupMapLockAcquisitionException(shardId, "Unable to obtain lock on the current Lookup map", null); - Engine.IndexResult lookupMapExceptionResult = new Engine.IndexResult(err, 0); - - IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenAnswer( - ir -> lookupMapExceptionResult - ); - when(shard.indexSettings()).thenReturn(indexSettings); - when(shard.shardId()).thenReturn(shardId); - when(shard.mapperService()).thenReturn(mock(MapperService.class)); - - UpdateHelper updateHelper = mock(UpdateHelper.class); - when(updateHelper.prepare(any(), eq(shard), any())).thenReturn( - new UpdateHelper.Result( - updateResponse, - randomBoolean() ? DocWriteResponse.Result.CREATED : DocWriteResponse.Result.UPDATED, - Collections.singletonMap("field", "value"), - Requests.INDEX_CONTENT_TYPE - ) - ); - - BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, RefreshPolicy.NONE, items.toArray(BulkItemRequest[]::new)); - - final CountDownLatch latch = new CountDownLatch(1); - Runnable runnable = () -> TransportShardBulkAction.performOnPrimary( - bulkShardRequest, - shard, - updateHelper, - threadPool::absoluteTimeInMillis, - new NoopMappingUpdatePerformer(), - listener -> listener.onResponse(null), - new LatchedActionListener<>(ActionTestUtils.assertNoFailureListener(result -> { - assertEquals(nItems, result.replicaRequest().items().length); - for (BulkItemRequest item : result.replicaRequest().items()) { - assertEquals(LookupMapLockAcquisitionException.class, item.getPrimaryResponse().getFailure().getCause().getClass()); - } - }), latch), - threadPool, - Names.WRITE - ); - - // execute the runnable on a separate thread so that the infinite loop can be detected - new Thread(runnable).start(); - - // timeout the request in 10 seconds if there is an infinite loop - assertTrue(latch.await(10, TimeUnit.SECONDS)); - - items.forEach(item -> { - assertEquals(item.getPrimaryResponse().getFailure().getCause().getClass(), LookupMapLockAcquisitionException.class); - - // this assertion is based on the assumption that all bulk item requests are updates and are hence calling - // UpdateRequest::prepareRequest - UpdateRequest updateRequest = (UpdateRequest) item.request(); - verify(updateHelper, times(retryCount + 1)).prepare(eq(updateRequest), any(IndexShard.class), any(LongSupplier.class)); - }); - } - public void testForceExecutionOnRejectionAfterMappingUpdate() throws Exception { TestThreadPool rejectingThreadPool = new TestThreadPool( "TransportShardBulkActionTests#testForceExecutionOnRejectionAfterMappingUpdate", diff --git a/server/src/test/java/org/opensearch/index/engine/CompositeIndexWriterForAppendTests.java b/server/src/test/java/org/opensearch/index/engine/CompositeIndexWriterForAppendTests.java index 7625bc5945e5a..dab474643eb56 100644 --- a/server/src/test/java/org/opensearch/index/engine/CompositeIndexWriterForAppendTests.java +++ b/server/src/test/java/org/opensearch/index/engine/CompositeIndexWriterForAppendTests.java @@ -41,6 +41,9 @@ import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class CompositeIndexWriterForAppendTests extends CriteriaBasedCompositeIndexWriterBaseTests { @@ -66,7 +69,12 @@ public void testGetIndexWriterWithRotatingMapAlwaysPutWriterInCurrentMap() throw CompositeIndexWriter.DisposableIndexWriter disposableIndexWriter; while (numOps > 0) { disposableIndexWriter = liveIndexWriterDeletesMap.get() - .computeIndexWriterIfAbsentForCriteria("200", this::createChildWriterFactory, new ShardId("foo", "_na_", 1)); + .computeIndexWriterIfAbsentForCriteria( + "200", + this::createChildWriterFactory, + new ShardId("foo", "_na_", 1), + MAX_NUMBER_OF_RETRIES + ); assertNotNull(disposableIndexWriter); assertFalse(disposableIndexWriter.getLookupMap().isClosed()); disposableIndexWriter.getIndexWriter().close(); @@ -130,7 +138,12 @@ public void testConcurrentComputeIndexWriterWithMapRotation() throws Exception { while (stopped.get() == false) { try { CompositeIndexWriter.LiveIndexWriterDeletesMap currentMap = mapRef.get(); - currentMap.computeIndexWriterIfAbsentForCriteria("test-criteria", supplier, new ShardId("foo", "_na_", 1)); + currentMap.computeIndexWriterIfAbsentForCriteria( + "test-criteria", + supplier, + new ShardId("foo", "_na_", 1), + MAX_NUMBER_OF_RETRIES + ); computeCount.incrementAndGet(); indexedDocs.release(); } catch (Exception e) { @@ -181,12 +194,38 @@ public void testUnableToObtainLockOnActiveLookupWhenWriteLockDuringIndexing() th expectThrows( LookupMapLockAcquisitionException.class, - () -> map.computeIndexWriterIfAbsentForCriteria("200", this::createChildWriterFactory, new ShardId("foo", "_na_", 1)) + () -> map.computeIndexWriterIfAbsentForCriteria( + "200", + this::createChildWriterFactory, + new ShardId("foo", "_na_", 1), + MAX_NUMBER_OF_RETRIES + ) ); releaseWriteLockLatch.countDown(); writer.join(); } + public void testMaxRetryCountWhenWriteLockDuringIndexing() throws IOException, InterruptedException { + CompositeIndexWriter.LiveIndexWriterDeletesMap map = new CompositeIndexWriter.LiveIndexWriterDeletesMap(); + map.current = mock(CompositeIndexWriter.CriteriaBasedIndexWriterLookup.class); + CompositeIndexWriter.CriteriaBasedIndexWriterLookup.CriteriaBasedWriterLock writerLock = mock( + CompositeIndexWriter.CriteriaBasedIndexWriterLookup.CriteriaBasedWriterLock.class + ); + map.current.mapReadLock = writerLock; + when(map.current.mapReadLock.tryAcquire()).thenReturn(null); + expectThrows( + LookupMapLockAcquisitionException.class, + () -> map.computeIndexWriterIfAbsentForCriteria( + "200", + this::createChildWriterFactory, + new ShardId("foo", "_na_", 1), + MAX_NUMBER_OF_RETRIES + ) + ); + + verify(writerLock, times(MAX_NUMBER_OF_RETRIES)).tryAcquire(); + } + public void testConcurrentIndexingDuringRefresh() throws IOException, InterruptedException { CompositeIndexWriter compositeIndexWriter = new CompositeIndexWriter( diff --git a/server/src/test/java/org/opensearch/index/engine/CriteriaBasedCompositeIndexWriterBaseTests.java b/server/src/test/java/org/opensearch/index/engine/CriteriaBasedCompositeIndexWriterBaseTests.java index 1742f4e481398..d745cd48fd70d 100644 --- a/server/src/test/java/org/opensearch/index/engine/CriteriaBasedCompositeIndexWriterBaseTests.java +++ b/server/src/test/java/org/opensearch/index/engine/CriteriaBasedCompositeIndexWriterBaseTests.java @@ -103,6 +103,7 @@ public class CriteriaBasedCompositeIndexWriterBaseTests extends OpenSearchTestCa protected final AllocationId allocationId = AllocationId.newInitializing(); protected final NumericDocValuesField softDeletesField = Lucene.newSoftDeletesField(); public static final String DEFAULT_CRITERIA = "testGroupingCriteria"; + public static final int MAX_NUMBER_OF_RETRIES = 20; protected IndexWriterFactory indexWriterFactory; protected static ParsedDocument createParsedDoc(String id, String routing, String groupingCriteria) {