From d41ab24a4b8fc17245ac544616c71499a994bc9d Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Sun, 22 Mar 2026 08:02:03 +0530 Subject: [PATCH 1/9] Add plugin for composite engine Signed-off-by: Arpit Bandejiya Co-authored-by: Bukhtawar Khan --- .gitignore | 1 + libs/concurrent-queue/build.gradle | 33 ++ .../common/queue/ConcurrentQueue.java | 140 +++++++++ .../org/opensearch/common/queue/Lockable.java | 34 ++ .../common/queue/LockableConcurrentQueue.java | 79 +++++ .../opensearch/common/queue/package-info.java | 14 + .../common/queue/ConcurrentQueueTests.java | 163 ++++++++++ .../queue/LockableConcurrentQueueTests.java | 218 +++++++++++++ sandbox/plugins/composite-engine/README.md | 32 ++ sandbox/plugins/composite-engine/build.gradle | 18 ++ .../composite/CompositeDataFormat.java | 82 +++++ .../composite/CompositeDocumentInput.java | 111 +++++++ .../composite/CompositeEnginePlugin.java | 210 +++++++++++++ .../CompositeIndexingExecutionEngine.java | 291 ++++++++++++++++++ .../opensearch/composite/CompositeWriter.java | 207 +++++++++++++ .../opensearch/composite/RowIdGenerator.java | 64 ++++ .../opensearch/composite/package-info.java | 17 + .../composite/CompositeDataFormatTests.java | 98 ++++++ .../CompositeDocumentInputTests.java | 224 ++++++++++++++ .../composite/CompositeEnginePluginTests.java | 211 +++++++++++++ ...CompositeIndexingExecutionEngineTests.java | 215 +++++++++++++ .../composite/CompositeTestHelper.java | 233 ++++++++++++++ .../composite/CompositeWriterTests.java | 98 ++++++ ...ataformatAwareLockableWriterPoolTests.java | 137 +++++++++ .../composite/RowIdGeneratorTests.java | 53 ++++ server/build.gradle | 1 + .../index/engine/dataformat/DataFormat.java | 24 +- .../engine/dataformat/DataFormatPlugin.java | 10 +- .../DataformatAwareLockableWriterPool.java | 166 ++++++++++ .../dataformat/DataFormatPluginTests.java | 23 +- 30 files changed, 3188 insertions(+), 19 deletions(-) create mode 100644 libs/concurrent-queue/build.gradle create mode 100644 libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java create mode 100644 libs/concurrent-queue/src/main/java/org/opensearch/common/queue/Lockable.java create mode 100644 libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java create mode 100644 libs/concurrent-queue/src/main/java/org/opensearch/common/queue/package-info.java create mode 100644 libs/concurrent-queue/src/test/java/org/opensearch/common/queue/ConcurrentQueueTests.java create mode 100644 libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockableConcurrentQueueTests.java create mode 100644 sandbox/plugins/composite-engine/README.md create mode 100644 sandbox/plugins/composite-engine/build.gradle create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDataFormat.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java create mode 100644 sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/package-info.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDataFormatTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java create mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java create mode 100644 server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java diff --git a/.gitignore b/.gitignore index 83eff29224279..4662ff4c5a4d1 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,7 @@ .claude CLAUDE.md .cursor* +.kiro* # intellij files .idea/ diff --git a/libs/concurrent-queue/build.gradle b/libs/concurrent-queue/build.gradle new file mode 100644 index 0000000000000..6a4cd5c909bb8 --- /dev/null +++ b/libs/concurrent-queue/build.gradle @@ -0,0 +1,33 @@ +/* + * 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. + */ + +/* + * Shared concurrent queue utilities for the composite indexing engine. + * No external dependencies — pure Java concurrency primitives. + */ + +dependencies { + /******* + * !!!! NO RUNTIME DEPENDENCIES !!!! + *******/ + + testImplementation "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" + testImplementation "junit:junit:${versions.junit}" + testImplementation "org.hamcrest:hamcrest:${versions.hamcrest}" + + testImplementation(project(":test:framework")) { + exclude group: 'org.opensearch', module: 'opensearch-concurrent-queue' + } +} + +testingConventions.enabled = true + +tasks.named('forbiddenApisMain').configure { + // :libs:opensearch-concurrent-queue does not depend on server + replaceSignatureFiles 'jdk-signatures' +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java new file mode 100644 index 0000000000000..f7856c9464f54 --- /dev/null +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java @@ -0,0 +1,140 @@ +/* + * 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.common.queue; + +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Predicate; +import java.util.function.Supplier; + +/** + * A striped concurrent queue that distributes entries across multiple internal + * queues using thread-affinity-based hashing. This reduces contention by allowing + * concurrent threads to operate on different stripes without blocking each other. + * + * @param the type of elements held in this queue + * @opensearch.experimental + */ +public final class ConcurrentQueue { + + static final int MIN_CONCURRENCY = 1; + static final int MAX_CONCURRENCY = 256; + + private final int concurrency; + private final Lock[] locks; + private final Queue[] queues; + private final Supplier> queueSupplier; + + ConcurrentQueue(Supplier> queueSupplier, int concurrency) { + if (concurrency < MIN_CONCURRENCY || concurrency > MAX_CONCURRENCY) { + throw new IllegalArgumentException( + "concurrency must be in [" + MIN_CONCURRENCY + ", " + MAX_CONCURRENCY + "], got " + concurrency + ); + } + this.concurrency = concurrency; + this.queueSupplier = queueSupplier; + locks = new Lock[concurrency]; + @SuppressWarnings({ "rawtypes", "unchecked" }) + Queue[] queues = new Queue[concurrency]; + this.queues = queues; + for (int i = 0; i < concurrency; ++i) { + locks[i] = new ReentrantLock(); + queues[i] = queueSupplier.get(); + } + } + + void add(T entry) { + // Seed the order in which to look at entries based on the current thread. This helps distribute + // entries across queues and gives a bit of thread affinity between entries and threads, which + // can't hurt. + final int threadHash = Thread.currentThread().hashCode() & 0xFFFF; + for (int i = 0; i < concurrency; ++i) { + final int index = (threadHash + i) % concurrency; + final Lock lock = locks[index]; + final Queue queue = queues[index]; + if (lock.tryLock()) { + try { + queue.add(entry); + return; + } finally { + lock.unlock(); + } + } + } + final int index = threadHash % concurrency; + final Lock lock = locks[index]; + final Queue queue = queues[index]; + lock.lock(); + try { + queue.add(entry); + } finally { + lock.unlock(); + } + } + + T poll(Predicate predicate) { + final int threadHash = Thread.currentThread().hashCode() & 0xFFFF; + for (int i = 0; i < concurrency; ++i) { + final int index = (threadHash + i) % concurrency; + final Lock lock = locks[index]; + final Queue queue = queues[index]; + if (lock.tryLock()) { + try { + Iterator it = queue.iterator(); + while (it.hasNext()) { + T entry = it.next(); + if (predicate.test(entry)) { + it.remove(); + return entry; + } + } + } finally { + lock.unlock(); + } + } + } + for (int i = 0; i < concurrency; ++i) { + final int index = (threadHash + i) % concurrency; + final Lock lock = locks[index]; + final Queue queue = queues[index]; + lock.lock(); + try { + Iterator it = queue.iterator(); + while (it.hasNext()) { + T entry = it.next(); + if (predicate.test(entry)) { + it.remove(); + return entry; + } + } + } finally { + lock.unlock(); + } + } + return null; + } + + boolean remove(T entry) { + for (int i = 0; i < concurrency; ++i) { + final Lock lock = locks[i]; + final Queue queue = queues[i]; + lock.lock(); + try { + if (queue.remove(entry)) { + return true; + } + } finally { + lock.unlock(); + } + } + return false; + } +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/Lockable.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/Lockable.java new file mode 100644 index 0000000000000..7d1f162190f2a --- /dev/null +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/Lockable.java @@ -0,0 +1,34 @@ +/* + * 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.common.queue; + +/** + * A minimal locking contract for objects managed by a {@link LockableConcurrentQueue}. + * + * @opensearch.experimental + */ +public interface Lockable { + + /** + * Acquires the lock. + */ + void lock(); + + /** + * Attempts to acquire the lock without blocking. + * + * @return {@code true} if the lock was acquired + */ + boolean tryLock(); + + /** + * Releases the lock. + */ + void unlock(); +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java new file mode 100644 index 0000000000000..8d03a73fde08e --- /dev/null +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java @@ -0,0 +1,79 @@ +/* + * 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.common.queue; + +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +/** + * A concurrent queue wrapper that adds lock-and-poll / add-and-unlock semantics + * on top of {@link ConcurrentQueue}. Entries must implement {@link Lockable} so that + * they can be atomically locked when polled and unlocked when returned. + *

+ * This is used by the composite writer pool to ensure that a writer is locked + * before it is handed out and unlocked when it is returned. + * + * @param the type of lockable elements held in this queue + * @opensearch.experimental + */ +public final class LockableConcurrentQueue { + + private final ConcurrentQueue queue; + private final AtomicInteger addAndUnlockCounter = new AtomicInteger(); + + /** + * Creates a new lockable concurrent queue. + * + * @param queueSupplier supplier for the underlying queue instances + * @param concurrency the concurrency level (number of stripes) + */ + public LockableConcurrentQueue(Supplier> queueSupplier, int concurrency) { + this.queue = new ConcurrentQueue<>(queueSupplier, concurrency); + } + + /** + * Lock an entry, and poll it from the queue, in that order. If no entry can be found and locked, + * {@code null} is returned. + */ + public T lockAndPoll() { + int addAndUnlockCount; + do { + addAndUnlockCount = addAndUnlockCounter.get(); + T entry = queue.poll(Lockable::tryLock); + if (entry != null) { + return entry; + } + // If an entry has been added to the queue in the meantime, try again. + } while (addAndUnlockCount != addAndUnlockCounter.get()); + + return null; + } + + /** + * Remove an entry from the queue. + * + * @param entry the entry to remove + * @return {@code true} if the entry was removed + */ + public boolean remove(T entry) { + return queue.remove(entry); + } + + /** + * Add an entry to the queue and unlock it, in that order. + * + * @param entry the entry to add and unlock + */ + public void addAndUnlock(T entry) { + queue.add(entry); + entry.unlock(); + addAndUnlockCounter.incrementAndGet(); + } +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/package-info.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/package-info.java new file mode 100644 index 0000000000000..8c3c36b667891 --- /dev/null +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Concurrent queue utilities for the composite indexing engine. + * + * @opensearch.experimental + */ +package org.opensearch.common.queue; diff --git a/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/ConcurrentQueueTests.java b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/ConcurrentQueueTests.java new file mode 100644 index 0000000000000..96de24bd5e5ba --- /dev/null +++ b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/ConcurrentQueueTests.java @@ -0,0 +1,163 @@ +/* + * 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.common.queue; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.LinkedList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Tests for {@link ConcurrentQueue}. + */ +public class ConcurrentQueueTests extends OpenSearchTestCase { + + public void testAddAndPollSingleThread() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 1); + queue.add("a"); + queue.add("b"); + assertEquals("a", queue.poll(e -> true)); + assertEquals("b", queue.poll(e -> true)); + assertNull(queue.poll(e -> true)); + } + + public void testPollWithPredicateFiltering() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 1); + queue.add(1); + queue.add(2); + queue.add(3); + // Poll only even numbers + assertEquals(Integer.valueOf(2), queue.poll(n -> n % 2 == 0)); + // Remaining: 1, 3 + assertNull(queue.poll(n -> n % 2 == 0)); + assertEquals(Integer.valueOf(1), queue.poll(e -> true)); + assertEquals(Integer.valueOf(3), queue.poll(e -> true)); + } + + public void testPollReturnsNullOnEmpty() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 4); + assertNull(queue.poll(e -> true)); + } + + public void testPollPredicateAlwaysFalse() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 2); + queue.add("a"); + assertNull(queue.poll(e -> false)); + // Entry should still be there + assertEquals("a", queue.poll(e -> true)); + } + + public void testRemoveExistingEntry() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 2); + queue.add("a"); + queue.add("b"); + assertTrue(queue.remove("a")); + assertEquals("b", queue.poll(e -> true)); + assertNull(queue.poll(e -> true)); + } + + public void testRemoveNonExistentEntry() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 2); + queue.add("a"); + assertFalse(queue.remove("z")); + assertEquals("a", queue.poll(e -> true)); + } + + public void testRemoveFromEmpty() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 1); + assertFalse(queue.remove("a")); + } + + public void testConcurrencyBoundsLow() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new ConcurrentQueue<>(LinkedList::new, 0)); + assertTrue(e.getMessage().contains("concurrency must be in")); + } + + public void testConcurrencyBoundsHigh() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new ConcurrentQueue<>(LinkedList::new, 257)); + assertTrue(e.getMessage().contains("concurrency must be in")); + } + + public void testMinConcurrency() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, ConcurrentQueue.MIN_CONCURRENCY); + queue.add("a"); + assertEquals("a", queue.poll(e -> true)); + } + + public void testMaxConcurrency() { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, ConcurrentQueue.MAX_CONCURRENCY); + queue.add("a"); + assertEquals("a", queue.poll(e -> true)); + } + + public void testMultipleStripes() { + // With higher concurrency, entries distribute across stripes + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 8); + int count = 100; + for (int i = 0; i < count; i++) { + queue.add(i); + } + AtomicInteger polled = new AtomicInteger(); + Integer entry; + while ((entry = queue.poll(e -> true)) != null) { + polled.incrementAndGet(); + } + assertEquals(count, polled.get()); + } + + public void testConcurrentAddAndPoll() throws Exception { + ConcurrentQueue queue = new ConcurrentQueue<>(LinkedList::new, 4); + int numThreads = 4; + int itemsPerThread = 250; + CyclicBarrier barrier = new CyclicBarrier(numThreads * 2); + CountDownLatch addLatch = new CountDownLatch(numThreads); + CountDownLatch pollLatch = new CountDownLatch(numThreads); + AtomicInteger totalPolled = new AtomicInteger(); + + // Producer threads + for (int t = 0; t < numThreads; t++) { + final int threadId = t; + new Thread(() -> { + try { + barrier.await(); + for (int i = 0; i < itemsPerThread; i++) { + queue.add(threadId * itemsPerThread + i); + } + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + addLatch.countDown(); + } + }).start(); + } + + // Consumer threads + for (int t = 0; t < numThreads; t++) { + new Thread(() -> { + try { + barrier.await(); + addLatch.await(); // Wait for all adds to complete + Integer item; + while ((item = queue.poll(e -> true)) != null) { + totalPolled.incrementAndGet(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + pollLatch.countDown(); + } + }).start(); + } + + pollLatch.await(); + assertEquals(numThreads * itemsPerThread, totalPolled.get()); + } +} diff --git a/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockableConcurrentQueueTests.java b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockableConcurrentQueueTests.java new file mode 100644 index 0000000000000..b56986cdcb484 --- /dev/null +++ b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockableConcurrentQueueTests.java @@ -0,0 +1,218 @@ +/* + * 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.common.queue; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.LinkedList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Tests for {@link LockableConcurrentQueue}. + */ +public class LockableConcurrentQueueTests extends OpenSearchTestCase { + + /** + * A simple lockable entry for testing. + */ + static class LockableEntry implements Lockable { + final String id; + private final ReentrantLock delegate = new ReentrantLock(); + + LockableEntry(String id) { + this.id = id; + } + + @Override + public void lock() { + delegate.lock(); + } + + @Override + public boolean tryLock() { + return delegate.tryLock(); + } + + @Override + public void unlock() { + delegate.unlock(); + } + + boolean isHeldByCurrentThread() { + return delegate.isHeldByCurrentThread(); + } + + boolean isLocked() { + return delegate.isLocked(); + } + + @Override + public String toString() { + return "LockableEntry{" + id + "}"; + } + } + + /** Helper: lock the entry, add it to the queue, which unlocks it. */ + private static void seedEntry(LockableConcurrentQueue queue, LockableEntry entry) { + entry.lock(); + queue.addAndUnlock(entry); + } + + public void testLockAndPollReturnsLockedEntry() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + LockableEntry entry = new LockableEntry("a"); + seedEntry(queue, entry); + + LockableEntry polled = queue.lockAndPoll(); + assertNotNull(polled); + assertSame(entry, polled); + assertTrue(polled.isHeldByCurrentThread()); + polled.unlock(); + } + + public void testLockAndPollReturnsNullOnEmpty() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + assertNull(queue.lockAndPoll()); + } + + public void testAddAndUnlockMakesEntryAvailable() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + LockableEntry entry = new LockableEntry("a"); + entry.lock(); + queue.addAndUnlock(entry); + // Entry should be unlocked after addAndUnlock + assertFalse(entry.isLocked()); + + LockableEntry polled = queue.lockAndPoll(); + assertNotNull(polled); + assertTrue(polled.isHeldByCurrentThread()); + polled.unlock(); + } + + public void testRemoveEntry() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + LockableEntry entry = new LockableEntry("a"); + seedEntry(queue, entry); + + assertTrue(queue.remove(entry)); + assertNull(queue.lockAndPoll()); + } + + public void testRemoveNonExistentEntry() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + LockableEntry entry = new LockableEntry("a"); + assertFalse(queue.remove(entry)); + } + + public void testMultipleEntries() { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 2); + LockableEntry a = new LockableEntry("a"); + LockableEntry b = new LockableEntry("b"); + LockableEntry c = new LockableEntry("c"); + + seedEntry(queue, a); + seedEntry(queue, b); + seedEntry(queue, c); + + int count = 0; + LockableEntry polled; + while ((polled = queue.lockAndPoll()) != null) { + assertTrue(polled.isHeldByCurrentThread()); + polled.unlock(); + count++; + } + assertEquals(3, count); + } + + public void testLockAndPollSkipsAlreadyLockedEntries() throws Exception { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 1); + LockableEntry a = new LockableEntry("a"); + LockableEntry b = new LockableEntry("b"); + + seedEntry(queue, a); + seedEntry(queue, b); + + // Lock 'a' from a different thread so tryLock fails for the current thread + CountDownLatch locked = new CountDownLatch(1); + CountDownLatch release = new CountDownLatch(1); + Thread locker = new Thread(() -> { + a.lock(); + locked.countDown(); + try { + release.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + a.unlock(); + } + }); + locker.start(); + locked.await(); + + try { + LockableEntry polled = queue.lockAndPoll(); + assertNotNull(polled); + assertSame(b, polled); + assertTrue(polled.isHeldByCurrentThread()); + polled.unlock(); + } finally { + release.countDown(); + locker.join(); + } + } + + public void testConcurrentLockAndPollAndAddAndUnlock() throws Exception { + LockableConcurrentQueue queue = new LockableConcurrentQueue<>(LinkedList::new, 4); + int numEntries = 20; + for (int i = 0; i < numEntries; i++) { + LockableEntry entry = new LockableEntry("entry-" + i); + seedEntry(queue, entry); + } + + int numThreads = 4; + int opsPerThread = 50; + CyclicBarrier barrier = new CyclicBarrier(numThreads); + CountDownLatch latch = new CountDownLatch(numThreads); + AtomicInteger pollCount = new AtomicInteger(); + + for (int t = 0; t < numThreads; t++) { + new Thread(() -> { + try { + barrier.await(); + for (int i = 0; i < opsPerThread; i++) { + LockableEntry entry = queue.lockAndPoll(); + if (entry != null) { + pollCount.incrementAndGet(); + assertTrue(entry.isHeldByCurrentThread()); + // Return it — entry is already locked by lockAndPoll + queue.addAndUnlock(entry); + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + latch.countDown(); + } + }).start(); + } + + latch.await(); + // All entries should still be in the queue (returned after use) + int remaining = 0; + LockableEntry entry; + while ((entry = queue.lockAndPoll()) != null) { + entry.unlock(); + remaining++; + } + assertEquals(numEntries, remaining); + } +} diff --git a/sandbox/plugins/composite-engine/README.md b/sandbox/plugins/composite-engine/README.md new file mode 100644 index 0000000000000..82a8fdfb44010 --- /dev/null +++ b/sandbox/plugins/composite-engine/README.md @@ -0,0 +1,32 @@ +# composite-engine + +Sandbox plugin that orchestrates multi-format indexing across multiple data format engines behind a single `IndexingExecutionEngine` interface. + +## What it does + +1. During `loadExtensions`, discovers all `DataFormatPlugin` implementations (e.g., Parquet, Arrow) via the `ExtensiblePlugin` SPI. +2. When composite indexing is enabled for an index, creates a `CompositeIndexingExecutionEngine` that delegates writes, refresh, and file management to each per-format engine. +3. Routes fields to the appropriate data format based on `FieldTypeCapabilities` declared by each format. +4. Manages a pool of `CompositeWriter` instances for concurrent indexing with lock-based checkout/release semantics. + +## Index settings + +| Setting | Default | Description | +|---|---|---| +| `index.composite.enabled` | `false` | Activates composite indexing for the index | +| `index.composite.primary_data_format` | `"lucene"` | The authoritative format used for merge operations | +| `index.composite.secondary_data_formats` | `[]` | Additional formats that receive writes alongside the primary | + +## How it fits in + +Format plugins (e.g., Parquet) extend this plugin by declaring `extendedPlugins = ['composite-engine']` in their `build.gradle` and implementing `DataFormatPlugin`. The `ExtensiblePlugin` SPI discovers them automatically during node bootstrap. + +## Key classes + +- **`CompositeEnginePlugin`** — The `ExtensiblePlugin` entry point. Discovers format plugins, validates settings, and creates the composite engine. +- **`CompositeIndexingExecutionEngine`** — Orchestrates indexing across primary and secondary format engines. +- **`CompositeDataFormat`** — A `DataFormat` that wraps multiple per-format instances. +- **`CompositeDocumentInput`** — Routes field additions to the appropriate per-format `DocumentInput` based on field type capabilities. +- **`CompositeWriter`** — A composite `Writer` that delegates write, flush, and sync to each per-format writer. +- **`CompositeDataFormatWriterPool`** — Thread-safe pool of `CompositeWriter` instances with lock-based checkout/release. +- **`RowIdGenerator`** — Generates monotonically increasing row IDs for cross-format document synchronization. diff --git a/sandbox/plugins/composite-engine/build.gradle b/sandbox/plugins/composite-engine/build.gradle new file mode 100644 index 0000000000000..527e2a904fe52 --- /dev/null +++ b/sandbox/plugins/composite-engine/build.gradle @@ -0,0 +1,18 @@ +/* + * 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. + */ + +opensearchplugin { + description = 'Composite indexing engine plugin that orchestrates multi-format indexing across multiple data format engines.' + classname = 'org.opensearch.composite.CompositeEnginePlugin' +} + +dependencies { + api project(':libs:opensearch-concurrent-queue') + compileOnly project(':server') + testImplementation project(':test:framework') +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDataFormat.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDataFormat.java new file mode 100644 index 0000000000000..2633ad0f30330 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDataFormat.java @@ -0,0 +1,82 @@ +/* + * 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.composite; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.FieldTypeCapabilities; + +import java.util.List; +import java.util.Objects; +import java.util.Set; + +/** + * A composite {@link DataFormat} that wraps multiple per-format {@link DataFormat} instances. + * Each constituent format retains its own {@link FieldTypeCapabilities} — field routing is + * handled per-format by {@link CompositeDocumentInput}, not by this class. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeDataFormat extends DataFormat { + + private final List dataFormats; + + /** + * Constructs a CompositeDataFormat from the given list of data formats. + * + * @param dataFormats the constituent data formats + */ + public CompositeDataFormat(List dataFormats) { + this.dataFormats = List.copyOf(Objects.requireNonNull(dataFormats, "dataFormats must not be null")); + } + + /** + * Constructs an empty CompositeDataFormat with no constituent formats. + */ + public CompositeDataFormat() { + this.dataFormats = List.of(); + } + + /** + * Returns the list of constituent data formats. + * + * @return the data formats + */ + public List getDataFormats() { + return dataFormats; + } + + @Override + public String name() { + return "composite"; + } + + @Override + public long priority() { + // In case some other format can independently support, + // the composite format should have the lowest priority + return Long.MIN_VALUE; + } + + @Override + public Set supportedFields() { + // Union of all constituent formats' supported fields + // TODO:: Post the changes done in mappings, we will relook this + if (dataFormats.isEmpty()) { + return Set.of(); + } + return dataFormats.get(0).supportedFields(); + } + + @Override + public String toString() { + return "CompositeDataFormat{dataFormats=" + dataFormats + '}'; + } +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java new file mode 100644 index 0000000000000..ff7ec0593249d --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java @@ -0,0 +1,111 @@ +/* + * 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.composite; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DocumentInput; +import org.opensearch.index.mapper.MappedFieldType; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A composite {@link DocumentInput} that wraps one {@link DocumentInput} per registered + * data format and broadcasts all field additions to every per-format input. + *

+ * Metadata operations ({@code setRowId}, {@code setVersion}, {@code setSeqNo}, + * {@code setPrimaryTerm}) and field additions are broadcast to all per-format inputs. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeDocumentInput implements DocumentInput>> { + + private final DocumentInput primaryDocumentInput; + private final DataFormat primaryFormat; + private final Map> secondaryDocumentInputs; + private final Runnable onClose; + + /** + * Constructs a CompositeDocumentInput with a primary format input and secondary format inputs. + * + * @param primaryFormat the primary data format + * @param primaryDocumentInput the document input for the primary format + * @param secondaryDocumentInputs a map of secondary data formats to their corresponding document inputs + * @param onClose callback invoked when this composite input is closed, typically to release the writer back to the pool + */ + public CompositeDocumentInput( + DataFormat primaryFormat, + DocumentInput primaryDocumentInput, + Map> secondaryDocumentInputs, + Runnable onClose + ) { + this.primaryFormat = Objects.requireNonNull(primaryFormat, "primaryFormat must not be null"); + this.primaryDocumentInput = Objects.requireNonNull(primaryDocumentInput, "primaryDocumentInput must not be null"); + this.secondaryDocumentInputs = Map.copyOf( + Objects.requireNonNull(secondaryDocumentInputs, "secondaryDocumentInputs must not be null") + ); + this.onClose = Objects.requireNonNull(onClose, "onClose must not be null"); + } + + @Override + public void addField(MappedFieldType fieldType, Object value) { + primaryDocumentInput.addField(fieldType, value); + for (DocumentInput input : secondaryDocumentInputs.values()) { + input.addField(fieldType, value); + } + } + + @Override + public void setRowId(String rowIdFieldName, long rowId) { + primaryDocumentInput.setRowId(rowIdFieldName, rowId); + for (DocumentInput input : secondaryDocumentInputs.values()) { + input.setRowId(rowIdFieldName, rowId); + } + } + + @Override + public List> getFinalInput() { + return null; + } + + @Override + public void close() { + onClose.run(); + } + + /** + * Returns the primary format's document input. + * + * @return the primary document input + */ + public DocumentInput getPrimaryInput() { + return primaryDocumentInput; + } + + /** + * Returns the primary data format. + * + * @return the primary data format + */ + public DataFormat getPrimaryFormat() { + return primaryFormat; + } + + /** + * Returns an unmodifiable map of secondary data formats to their document inputs. + * + * @return the secondary inputs + */ + public Map> getSecondaryInputs() { + return secondaryDocumentInputs; + } +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java new file mode 100644 index 0000000000000..2d86c07f96cd4 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java @@ -0,0 +1,210 @@ +/* + * 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.composite; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexModule; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DataFormatPlugin; +import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; +import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.plugins.ExtensiblePlugin; +import org.opensearch.plugins.Plugin; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Sandbox plugin that provides a {@link CompositeIndexingExecutionEngine} for + * orchestrating multi-format indexing. Discovers {@link DataFormatPlugin} instances + * during node bootstrap via the {@link ExtensiblePlugin} SPI and creates a composite + * engine when composite indexing is enabled for an index. + *

+ * Registers two index settings: + *

    + *
  • {@code index.composite.enabled} — activates composite indexing (default {@code false})
  • + *
  • {@code index.composite.primary_data_format} — designates the primary format (default {@code "lucene"})
  • + *
+ *

+ * Format plugins (e.g., Parquet) extend this plugin by declaring + * {@code extendedPlugins = ['composite-engine']} in their {@code build.gradle} + * and implementing {@link DataFormatPlugin}. The {@link ExtensiblePlugin} SPI + * discovers them automatically during node bootstrap. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeEnginePlugin extends Plugin implements ExtensiblePlugin, DataFormatPlugin { + + private static final Logger logger = LogManager.getLogger(CompositeEnginePlugin.class); + + /** + * Index setting to enable composite indexing for an index. + * When {@code true}, the composite engine orchestrates writes across all registered data formats. + * Validates that the primary data format is non-empty when enabled. + */ + public static final Setting COMPOSITE_ENABLED = Setting.boolSetting( + "index.composite.enabled", + false, + new Setting.Validator<>() { + @Override + public void validate(Boolean value) {} + + @Override + public void validate(Boolean enabled, Map, Object> settings) { + if (enabled) { + String primary = (String) settings.get(PRIMARY_DATA_FORMAT); + if (primary == null || primary.isEmpty()) { + throw new IllegalArgumentException( + "[index.composite.enabled] requires [index.composite.primary_data_format] to be set" + ); + } + } + } + + @Override + public Iterator> settings() { + return List.>of(PRIMARY_DATA_FORMAT).iterator(); + } + }, + Setting.Property.IndexScope + ); + + /** + * Index setting that designates the primary data format for an index. + * The primary format is the authoritative format used for merge operations. + */ + public static final Setting PRIMARY_DATA_FORMAT = Setting.simpleString( + "index.composite.primary_data_format", + "lucene", + Setting.Property.IndexScope + ); + + /** + * Index setting that lists the secondary data formats for an index. + * Secondary formats receive writes alongside the primary but are not used + * as the merge authority. + */ + public static final Setting> SECONDARY_DATA_FORMATS = Setting.listSetting( + "index.composite.secondary_data_formats", + Collections.emptyList(), + s -> s, + Setting.Property.IndexScope + ); + + /** + * Discovered {@link DataFormatPlugin} instances keyed by format name. + * When multiple plugins declare the same format name, the one with the highest + * {@link DataFormat#priority()} is retained. + */ + private volatile Map dataFormatPlugins = Map.of(); + + /** Creates a new composite engine plugin. */ + public CompositeEnginePlugin() {} + + @Override + public void loadExtensions(ExtensionLoader loader) { + List formatPlugins = loader.loadExtensions(DataFormatPlugin.class); + Map registry = new HashMap<>(); + for (DataFormatPlugin plugin : formatPlugins) { + DataFormat format = plugin.getDataFormat(); + if (format == null) { + logger.warn("DataFormatPlugin [{}] returned null DataFormat, skipping", plugin.getClass().getName()); + continue; + } + String name = format.name(); + if (name == null || name.isBlank()) { + logger.warn("DataFormatPlugin [{}] returned a DataFormat with null/blank name, skipping", plugin.getClass().getName()); + continue; + } + DataFormatPlugin existing = registry.get(name); + if (existing != null) { + long existingPriority = existing.getDataFormat().priority(); + if (format.priority() <= existingPriority) { + logger.debug( + "Skipping DataFormatPlugin [{}] for format [{}] (priority {} <= existing {})", + plugin.getClass().getName(), + name, + format.priority(), + existingPriority + ); + continue; + } + logger.info( + "Replacing DataFormatPlugin for format [{}] (priority {} > existing {})", + name, + format.priority(), + existingPriority + ); + } + registry.put(name, plugin); + logger.info("Registered DataFormatPlugin [{}] for format [{}]", plugin.getClass().getName(), name); + } + this.dataFormatPlugins = Collections.unmodifiableMap(registry); + } + + @Override + public List> getSettings() { + return List.of(COMPOSITE_ENABLED, PRIMARY_DATA_FORMAT, SECONDARY_DATA_FORMATS); + } + + @Override + public void onIndexModule(IndexModule indexModule) { + Settings settings = indexModule.getSettings(); + boolean compositeEnabled = COMPOSITE_ENABLED.get(settings); + if (compositeEnabled == false) { + return; + } + + String primaryFormatName = PRIMARY_DATA_FORMAT.get(settings); + List secondaryFormatNames = SECONDARY_DATA_FORMATS.get(settings); + CompositeIndexingExecutionEngine.validateFormatsRegistered(dataFormatPlugins, primaryFormatName, secondaryFormatNames); + } + + @Override + public DataFormat getDataFormat() { + // TODO: Dataformat for Composite is per index, while this one talks about cluster level. Switching it off for now + return null; + } + + @Override + public IndexingExecutionEngine indexingEngine( + MapperService mapperService, + ShardPath shardPath, + IndexSettings indexSettings, + @Nullable DataformatAwareLockableWriterPool writerPool + ) { + Objects.requireNonNull(writerPool, "DataformatAwareLockableWriterPool is required for CompositeIndexingExecutionEngine"); + @SuppressWarnings("unchecked") + DataformatAwareLockableWriterPool compositeWriterPool = (DataformatAwareLockableWriterPool< + CompositeWriter>) writerPool; + return new CompositeIndexingExecutionEngine(dataFormatPlugins, indexSettings, mapperService, shardPath, compositeWriterPool); + } + + /** + * Returns the discovered data format plugins keyed by format name. + * + * @return unmodifiable map of format name to plugin + */ + public Map getDataFormatPlugins() { + return dataFormatPlugins; + } +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java new file mode 100644 index 0000000000000..bbfbb0e5c3531 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java @@ -0,0 +1,291 @@ +/* + * 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.composite; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DataFormatPlugin; +import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; +import org.opensearch.index.engine.dataformat.DocumentInput; +import org.opensearch.index.engine.dataformat.FileInfos; +import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; +import org.opensearch.index.engine.dataformat.Merger; +import org.opensearch.index.engine.dataformat.RefreshInput; +import org.opensearch.index.engine.dataformat.RefreshResult; +import org.opensearch.index.engine.dataformat.Writer; +import org.opensearch.index.engine.exec.Segment; +import org.opensearch.index.engine.exec.WriterFileSet; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.shard.ShardPath; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; + +/** + * A composite {@link IndexingExecutionEngine} that orchestrates indexing across + * multiple per-format engines behind a single interface. + *

+ * The engine delegates writer creation, refresh, file deletion, and document input + * creation to each per-format engine. A primary engine is designated based on the + * configured primary format name and is used for merge operations. + *

+ * The composite {@link DataFormat} exposed by this engine represents the union of + * all per-format supported field type capabilities. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeIndexingExecutionEngine implements IndexingExecutionEngine { + + private static final Logger logger = LogManager.getLogger(CompositeIndexingExecutionEngine.class); + + private final IndexingExecutionEngine primaryEngine; + private final List> secondaryEngines; + private final CompositeDataFormat compositeDataFormat; + private final DataformatAwareLockableWriterPool writerPool; + + /** + * Constructs a CompositeIndexingExecutionEngine by reading index settings to + * determine the primary and secondary data formats, validating that all configured + * formats are registered, and creating per-format engines via the discovered + * {@link DataFormatPlugin} instances. + *

+ * The primary engine is the authoritative format used for merge operations and + * commit coordination. Secondary engines receive writes alongside the primary but + * are not used as the merge authority. + *

+ * The pool is initialized with a writer supplier that creates {@link CompositeWriter} + * instances bound to this engine, eliminating the circular dependency between the + * engine and its writer pool. + * + * @param dataFormatPlugins the discovered data format plugins keyed by format name + * @param indexSettings the index settings containing composite configuration + * @param mapperService the mapper service for field mapping resolution + * @param shardPath the shard path for file storage + * @param writerPool the writer pool for managing composite writer instances + * @throws IllegalStateException if composite indexing is not enabled + * @throws IllegalArgumentException if any configured format is not registered + * @throws NullPointerException if writerPool is null + */ + public CompositeIndexingExecutionEngine( + Map dataFormatPlugins, + IndexSettings indexSettings, + MapperService mapperService, + ShardPath shardPath, + DataformatAwareLockableWriterPool writerPool + ) { + Objects.requireNonNull(dataFormatPlugins, "dataFormatPlugins must not be null"); + Objects.requireNonNull(indexSettings, "indexSettings must not be null"); + Objects.requireNonNull(writerPool, "writerPool must not be null"); + + Settings settings = indexSettings.getSettings(); + boolean compositeEnabled = CompositeEnginePlugin.COMPOSITE_ENABLED.get(settings); + if (compositeEnabled == false) { + throw new IllegalStateException("Composite indexing is not enabled for index [" + indexSettings.getIndex().getName() + "]"); + } + + String primaryFormatName = CompositeEnginePlugin.PRIMARY_DATA_FORMAT.get(settings); + List secondaryFormatNames = CompositeEnginePlugin.SECONDARY_DATA_FORMATS.get(settings); + + validateFormatsRegistered(dataFormatPlugins, primaryFormatName, secondaryFormatNames); + + List allFormats = new ArrayList<>(); + DataFormatPlugin primaryPlugin = dataFormatPlugins.get(primaryFormatName); + this.primaryEngine = primaryPlugin.indexingEngine(mapperService, shardPath, indexSettings, null); + allFormats.add(primaryPlugin.getDataFormat()); + + List> secondaries = new ArrayList<>(); + for (String secondaryName : secondaryFormatNames) { + DataFormatPlugin secondaryPlugin = dataFormatPlugins.get(secondaryName); + secondaries.add(secondaryPlugin.indexingEngine(mapperService, shardPath, indexSettings, null)); + allFormats.add(secondaryPlugin.getDataFormat()); + } + this.secondaryEngines = List.copyOf(secondaries); + + this.compositeDataFormat = new CompositeDataFormat(allFormats); + this.writerPool = writerPool; + + // Initialize the pool with a writer supplier now that the engine is fully constructed + AtomicLong writerGenerationCounter = new AtomicLong(0); + writerPool.initialize(() -> new CompositeWriter(this, writerGenerationCounter.getAndIncrement())); + } + + /** + * Validates that the primary and all secondary data format plugins are registered. + * + * @param dataFormatPlugins the discovered data format plugins keyed by format name + * @param primaryFormatName the configured primary format name + * @param secondaryFormatNames the configured secondary format names + * @throws IllegalArgumentException if any configured format is not registered + */ + static void validateFormatsRegistered( + Map dataFormatPlugins, + String primaryFormatName, + List secondaryFormatNames + ) { + if (dataFormatPlugins.containsKey(primaryFormatName) == false) { + throw new IllegalArgumentException( + "Primary data format [" + + primaryFormatName + + "] is not registered on this node. Available formats: " + + dataFormatPlugins.keySet() + ); + } + for (String secondaryName : secondaryFormatNames) { + if (secondaryName.equals(primaryFormatName)) { + throw new IllegalStateException( + "Secondary data format [" + secondaryName + "] is the same as primary :[" + primaryFormatName + "]" + ); + } + if (dataFormatPlugins.containsKey(secondaryName) == false) { + throw new IllegalArgumentException( + "Secondary data format [" + + secondaryName + + "] is not registered on this node. Available formats: " + + dataFormatPlugins.keySet() + ); + } + } + } + + @Override + public Writer createWriter(long writerGeneration) { + return new CompositeWriter(this, writerGeneration); + } + + @Override + public Merger getMerger() { + return primaryEngine.getMerger(); + } + + @Override + public RefreshResult refresh(RefreshInput refreshInput) throws IOException { + List dataFormatWriters = writerPool.checkoutAll(); + + // Mark each writer as flush-pending before flushing + for (CompositeWriter writer : dataFormatWriters) { + writer.setFlushPending(); + } + + List refreshedSegments = new ArrayList<>(refreshInput.existingSegments()); + List newSegmentList = new ArrayList<>(); + + logger.debug( + "Refreshing composite engine: flushing {} writers, existing segments={}", + dataFormatWriters.size(), + refreshedSegments.size() + ); + + // Flush each writer to disk and build segments from the file infos + for (CompositeWriter writer : dataFormatWriters) { + FileInfos fileInfos = writer.flush(); + Segment.Builder segmentBuilder = Segment.builder(writer.getWriterGeneration()); + boolean hasFiles = false; + for (Map.Entry entry : fileInfos.writerFilesMap().entrySet()) { + logger.debug( + "Writer gen={} flushed format=[{}] files={}", + writer.getWriterGeneration(), + entry.getKey().name(), + entry.getValue().files() + ); + segmentBuilder.addSearchableFiles(entry.getKey(), entry.getValue()); + hasFiles = true; + } + writer.close(); + if (hasFiles) { + newSegmentList.add(segmentBuilder.build()); + } + } + + if (newSegmentList.isEmpty()) { + logger.debug("No new segments produced from flush"); + return null; + } + + logger.debug("Produced {} new segments from flush", newSegmentList.size()); + refreshedSegments.addAll(newSegmentList); + + // Delegate refresh to each per-format engine + RefreshInput emptyInput = RefreshInput.builder().build(); + primaryEngine.refresh(emptyInput); + for (IndexingExecutionEngine engine : secondaryEngines) { + engine.refresh(emptyInput); + } + + return new RefreshResult(refreshedSegments); + } + + @Override + public CompositeDataFormat getDataFormat() { + return compositeDataFormat; + } + + @Override + public long getNativeBytesUsed() { + long total = primaryEngine.getNativeBytesUsed(); + for (IndexingExecutionEngine engine : secondaryEngines) { + total += engine.getNativeBytesUsed(); + } + return total; + } + + @Override + public void deleteFiles(Map> filesToDelete) throws IOException { + primaryEngine.deleteFiles(filesToDelete); + for (IndexingExecutionEngine engine : secondaryEngines) { + engine.deleteFiles(filesToDelete); + } + } + + @Override + public CompositeDocumentInput newDocumentInput() { + CompositeWriter writer = writerPool.getAndLock(); + DocumentInput primaryInput = primaryEngine.newDocumentInput(); + Map> secondaryInputMap = new LinkedHashMap<>(); + for (IndexingExecutionEngine engine : secondaryEngines) { + secondaryInputMap.put(engine.getDataFormat(), engine.newDocumentInput()); + } + return new CompositeDocumentInput(primaryEngine.getDataFormat(), primaryInput, secondaryInputMap, () -> { + assert writer.isFlushPending() == false && writer.isAborted() == false : "CompositeWriter has pending flush: " + + writer.isFlushPending() + + " aborted=" + + writer.isAborted(); + writerPool.releaseAndUnlock(writer); + }); + } + + /** + * Returns the primary delegate engine. + * + * @return the primary engine + */ + public IndexingExecutionEngine getPrimaryDelegate() { + return primaryEngine; + } + + /** + * Returns the secondary delegate engines. + * + * @return the secondary engines + */ + public List> getSecondaryDelegates() { + return secondaryEngines; + } + +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java new file mode 100644 index 0000000000000..fe1338d4df020 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java @@ -0,0 +1,207 @@ +/* + * 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.composite; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.queue.Lockable; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DocumentInput; +import org.opensearch.index.engine.dataformat.FileInfos; +import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; +import org.opensearch.index.engine.dataformat.WriteResult; +import org.opensearch.index.engine.dataformat.Writer; +import org.opensearch.index.engine.exec.WriterFileSet; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.locks.ReentrantLock; + +/** + * A composite {@link Writer} that wraps one {@link Writer} per registered data format + * and delegates write operations to each per-format writer. + *

+ * Constructed from a {@link CompositeIndexingExecutionEngine}, it iterates the engine's + * delegates to create per-format writers. The primary format's writer is always first + * in the {@code writers} list. A {@code postWrite} callback releases this writer back + * to the pool after each write cycle. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeWriter implements Writer, Lockable { + + private static final Logger logger = LogManager.getLogger(CompositeWriter.class); + + private final Map.Entry>> primaryWriter; + private final Map>> secondaryWritersByFormat; + private final ReentrantLock lock; + private final long writerGeneration; + private final RowIdGenerator rowIdGenerator; + private volatile boolean aborted; + private volatile boolean flushPending; + + /** + * Constructs a CompositeWriter from the given engine and writer generation. + *

+ * Creates a per-format {@link Writer} for each delegate engine (primary first, + * then secondaries). The {@code postWrite} callback releases this writer back + * to the engine's writer pool. + * + * @param engine the composite indexing execution engine + * @param writerGeneration the writer generation number + */ + @SuppressWarnings("unchecked") + public CompositeWriter(CompositeIndexingExecutionEngine engine, long writerGeneration) { + this.lock = new ReentrantLock(); + this.aborted = false; + this.flushPending = false; + this.writerGeneration = writerGeneration; + + IndexingExecutionEngine primaryDelegate = engine.getPrimaryDelegate(); + this.primaryWriter = new AbstractMap.SimpleImmutableEntry<>( + primaryDelegate.getDataFormat(), + (Writer>) primaryDelegate.createWriter(writerGeneration) + ); + + Map>> secondaries = new LinkedHashMap<>(); + for (IndexingExecutionEngine delegate : engine.getSecondaryDelegates()) { + secondaries.put(delegate.getDataFormat(), (Writer>) delegate.createWriter(writerGeneration)); + } + this.secondaryWritersByFormat = Map.copyOf(secondaries); + this.rowIdGenerator = new RowIdGenerator(CompositeWriter.class.getName()); + } + + @Override + public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { + // Write to primary first + WriteResult primaryResult = primaryWriter.getValue().addDoc(doc.getPrimaryInput()); + switch (primaryResult) { + case WriteResult.Success s -> logger.trace("Successfully added document in primary format [{}]", primaryWriter.getKey().name()); + case WriteResult.Failure f -> { + logger.debug("Failed to add document in primary format [{}]", primaryWriter.getKey().name()); + return primaryResult; + } + } + + // Then write to each secondary — keyed lookup by DataFormat (equals/hashCode based on name) + Map> secondaryInputs = doc.getSecondaryInputs(); + for (Map.Entry> inputEntry : secondaryInputs.entrySet()) { + DataFormat format = inputEntry.getKey(); + Writer> writer = secondaryWritersByFormat.get(format); + if (writer == null) { + logger.warn("No writer found for secondary format [{}], skipping", format.name()); + continue; + } + WriteResult result = writer.addDoc(inputEntry.getValue()); + switch (result) { + case WriteResult.Success s -> logger.trace("Successfully added document in secondary format [{}]", format.name()); + case WriteResult.Failure f -> { + logger.debug("Failed to add document in secondary format [{}]", format.name()); + return result; + } + } + } + + return primaryResult; + } + + @Override + public FileInfos flush() throws IOException { + FileInfos.Builder builder = FileInfos.builder(); + // Flush primary + Optional primaryWfs = primaryWriter.getValue().flush().getWriterFileSet(primaryWriter.getKey()); + primaryWfs.ifPresent(writerFileSet -> builder.putWriterFileSet(primaryWriter.getKey(), writerFileSet)); + // Flush secondaries + for (Writer> writer : secondaryWritersByFormat.values()) { + FileInfos fileInfos = writer.flush(); + // Iterate all format entries in the returned FileInfos + for (Map.Entry fileEntry : fileInfos.writerFilesMap().entrySet()) { + builder.putWriterFileSet(fileEntry.getKey(), fileEntry.getValue()); + } + } + return builder.build(); + } + + @Override + public void sync() throws IOException { + primaryWriter.getValue().sync(); + for (Writer> writer : secondaryWritersByFormat.values()) { + writer.sync(); + } + } + + @Override + public void close() throws IOException { + primaryWriter.getValue().close(); + for (Writer> writer : secondaryWritersByFormat.values()) { + writer.close(); + } + } + + /** + * Returns the writer generation number. + * + * @return the writer generation + */ + public long getWriterGeneration() { + return writerGeneration; + } + + /** + * Marks this writer as aborted. + */ + public void abort() { + this.aborted = true; + } + + /** + * Returns whether this writer has been aborted. + * + * @return {@code true} if aborted + */ + public boolean isAborted() { + return aborted; + } + + /** + * Marks this writer as having a pending flush. + */ + public void setFlushPending() { + this.flushPending = true; + } + + /** + * Returns whether this writer has a pending flush. + * + * @return {@code true} if a flush is pending + */ + public boolean isFlushPending() { + return flushPending; + } + + @Override + public void lock() { + lock.lock(); + } + + @Override + public boolean tryLock() { + return lock.tryLock(); + } + + @Override + public void unlock() { + lock.unlock(); + } +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java new file mode 100644 index 0000000000000..9f6c223ddb5db --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java @@ -0,0 +1,64 @@ +/* + * 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.composite; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * Generates monotonically increasing row IDs for cross-format document synchronization. + * Each writer instance gets its own {@code RowIdGenerator} so that row IDs are unique + * within a writer's segment scope. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class RowIdGenerator { + + private final String source; + private final AtomicLong counter; + + /** + * Constructs a RowIdGenerator with the given source identifier. + * + * @param source a human-readable label identifying the generator's owner (e.g. class name) + */ + public RowIdGenerator(String source) { + this.source = source; + this.counter = new AtomicLong(0); + } + + /** + * Returns the next row ID. + * + * @return the next monotonically increasing row ID + */ + public long nextRowId() { + return counter.getAndIncrement(); + } + + /** + * Returns the current row ID value without incrementing. + * + * @return the current row ID + */ + public long currentRowId() { + return counter.get(); + } + + /** + * Returns the source identifier for this generator. + * + * @return the source label + */ + public String getSource() { + return source; + } +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/package-info.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/package-info.java new file mode 100644 index 0000000000000..216fe2c5469ae --- /dev/null +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/package-info.java @@ -0,0 +1,17 @@ +/* + * 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. + */ + +/** + * Composite indexing engine plugin that orchestrates multi-format indexing + * across multiple data format engines. + * @opensearch.experimental + */ +@ExperimentalApi +package org.opensearch.composite; + +import org.opensearch.common.annotation.ExperimentalApi; diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDataFormatTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDataFormatTests.java new file mode 100644 index 0000000000000..b6be1f41767d9 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDataFormatTests.java @@ -0,0 +1,98 @@ +/* + * 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.composite; + +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.FieldTypeCapabilities; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.List; +import java.util.Set; + +/** + * Tests for {@link CompositeDataFormat}. + */ +public class CompositeDataFormatTests extends OpenSearchTestCase { + + public void testNameReturnsComposite() { + CompositeDataFormat format = new CompositeDataFormat(List.of(mockFormat("lucene", 1, Set.of()))); + assertEquals("composite", format.name()); + } + + public void testPriorityReturnsMinValue() { + CompositeDataFormat format = new CompositeDataFormat(List.of(mockFormat("lucene", 1, Set.of()))); + assertEquals(Long.MIN_VALUE, format.priority()); + } + + public void testDefaultConstructorReturnsEmptyFormats() { + CompositeDataFormat format = new CompositeDataFormat(); + assertTrue(format.getDataFormats().isEmpty()); + assertEquals(Set.of(), format.supportedFields()); + } + + public void testSupportedFieldsDelegatesToFirstFormat() { + FieldTypeCapabilities cap1 = new FieldTypeCapabilities("keyword", Set.of(FieldTypeCapabilities.Capability.FULL_TEXT_SEARCH)); + FieldTypeCapabilities cap2 = new FieldTypeCapabilities("integer", Set.of(FieldTypeCapabilities.Capability.COLUMNAR_STORAGE)); + DataFormat primary = mockFormat("lucene", 1, Set.of(cap1)); + DataFormat secondary = mockFormat("parquet", 2, Set.of(cap2)); + + CompositeDataFormat composite = new CompositeDataFormat(List.of(primary, secondary)); + // supportedFields() returns the first format's fields + assertEquals(Set.of(cap1), composite.supportedFields()); + } + + public void testSupportedFieldsEmptyWhenNoFormats() { + CompositeDataFormat composite = new CompositeDataFormat(List.of()); + assertEquals(Set.of(), composite.supportedFields()); + } + + public void testGetDataFormatsReturnsAllFormats() { + DataFormat f1 = mockFormat("lucene", 1, Set.of()); + DataFormat f2 = mockFormat("parquet", 2, Set.of()); + CompositeDataFormat composite = new CompositeDataFormat(List.of(f1, f2)); + assertEquals(2, composite.getDataFormats().size()); + assertSame(f1, composite.getDataFormats().get(0)); + assertSame(f2, composite.getDataFormats().get(1)); + } + + public void testGetDataFormatsIsUnmodifiable() { + CompositeDataFormat composite = new CompositeDataFormat(List.of(mockFormat("lucene", 1, Set.of()))); + expectThrows(UnsupportedOperationException.class, () -> composite.getDataFormats().add(mockFormat("x", 0, Set.of()))); + } + + public void testConstructorRejectsNull() { + expectThrows(NullPointerException.class, () -> new CompositeDataFormat(null)); + } + + public void testToStringContainsClassName() { + CompositeDataFormat composite = new CompositeDataFormat(List.of(mockFormat("lucene", 1, Set.of()))); + String str = composite.toString(); + assertTrue(str.contains("CompositeDataFormat")); + assertTrue(str.contains("dataFormats=")); + } + + private DataFormat mockFormat(String name, long priority, Set fields) { + return new DataFormat() { + @Override + public String name() { + return name; + } + + @Override + public long priority() { + return priority; + } + + @Override + public Set supportedFields() { + return fields; + } + }; + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java new file mode 100644 index 0000000000000..958532689fdd7 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java @@ -0,0 +1,224 @@ +/* + * 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.composite; + +import org.apache.lucene.search.Query; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DocumentInput; +import org.opensearch.index.engine.dataformat.FieldTypeCapabilities; +import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.mapper.TextSearchInfo; +import org.opensearch.index.mapper.ValueFetcher; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.search.lookup.SearchLookup; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Tests for {@link CompositeDocumentInput}. + */ +public class CompositeDocumentInputTests extends OpenSearchTestCase { + + public void testAddFieldBroadcastsToAllFormats() { + RecordingDocumentInput primaryInput = new RecordingDocumentInput(); + RecordingDocumentInput secondaryInput = new RecordingDocumentInput(); + + DataFormat primaryFormat = mockFormat("lucene", 1, Set.of()); + DataFormat secondaryFormat = mockFormat("parquet", 2, Set.of()); + + CompositeDocumentInput composite = new CompositeDocumentInput( + primaryFormat, + primaryInput, + Map.of(secondaryFormat, secondaryInput), + () -> {} + ); + + MappedFieldType keywordField = mockFieldType("keyword"); + composite.addField(keywordField, "value1"); + + assertEquals(1, primaryInput.addedFields.size()); + assertEquals(1, secondaryInput.addedFields.size()); + } + + public void testSetRowIdBroadcastsToAllInputs() { + RecordingDocumentInput primaryInput = new RecordingDocumentInput(); + RecordingDocumentInput secondary1 = new RecordingDocumentInput(); + RecordingDocumentInput secondary2 = new RecordingDocumentInput(); + + DataFormat primaryFormat = mockFormat("lucene", 1, Set.of()); + DataFormat secondaryFormat1 = mockFormat("parquet", 2, Set.of()); + DataFormat secondaryFormat2 = mockFormat("arrow", 3, Set.of()); + + Map> secondaries = new HashMap<>(); + secondaries.put(secondaryFormat1, secondary1); + secondaries.put(secondaryFormat2, secondary2); + + CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, primaryInput, secondaries, () -> {}); + + composite.setRowId("_row_id", 42L); + + assertEquals(1, primaryInput.rowIds.size()); + assertEquals(42L, (long) primaryInput.rowIds.get(0)); + assertEquals(1, secondary1.rowIds.size()); + assertEquals(42L, (long) secondary1.rowIds.get(0)); + assertEquals(1, secondary2.rowIds.size()); + assertEquals(42L, (long) secondary2.rowIds.get(0)); + } + + public void testGetFinalInputReturnsNull() { + CompositeDocumentInput composite = new CompositeDocumentInput( + mockFormat("lucene", 1, Set.of()), + new RecordingDocumentInput(), + Map.of(), + () -> {} + ); + assertNull(composite.getFinalInput()); + } + + public void testCloseInvokesOnCloseCallback() { + AtomicBoolean closed = new AtomicBoolean(false); + CompositeDocumentInput composite = new CompositeDocumentInput( + mockFormat("lucene", 1, Set.of()), + new RecordingDocumentInput(), + Map.of(), + () -> closed.set(true) + ); + + composite.close(); + assertTrue(closed.get()); + } + + public void testGetPrimaryInputReturnsPrimaryDocumentInput() { + RecordingDocumentInput primaryInput = new RecordingDocumentInput(); + CompositeDocumentInput composite = new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), primaryInput, Map.of(), () -> {}); + assertSame(primaryInput, composite.getPrimaryInput()); + } + + public void testGetPrimaryFormatReturnsPrimaryDataFormat() { + DataFormat primaryFormat = mockFormat("lucene", 1, Set.of()); + CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, new RecordingDocumentInput(), Map.of(), () -> {}); + assertSame(primaryFormat, composite.getPrimaryFormat()); + } + + public void testGetSecondaryInputsReturnsUnmodifiableMap() { + DataFormat secondaryFormat = mockFormat("parquet", 2, Set.of()); + RecordingDocumentInput secondaryInput = new RecordingDocumentInput(); + + CompositeDocumentInput composite = new CompositeDocumentInput( + mockFormat("lucene", 1, Set.of()), + new RecordingDocumentInput(), + Map.of(secondaryFormat, secondaryInput), + () -> {} + ); + + Map> secondaries = composite.getSecondaryInputs(); + assertEquals(1, secondaries.size()); + expectThrows( + UnsupportedOperationException.class, + () -> secondaries.put(mockFormat("x", 0, Set.of()), new RecordingDocumentInput()) + ); + } + + public void testConstructorRejectsNullPrimaryFormat() { + expectThrows(NullPointerException.class, () -> new CompositeDocumentInput(null, new RecordingDocumentInput(), Map.of(), () -> {})); + } + + public void testConstructorRejectsNullPrimaryInput() { + expectThrows( + NullPointerException.class, + () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), null, Map.of(), () -> {}) + ); + } + + public void testConstructorRejectsNullSecondaryInputs() { + expectThrows( + NullPointerException.class, + () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), null, () -> {}) + ); + } + + public void testConstructorRejectsNullOnClose() { + expectThrows( + NullPointerException.class, + () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), Map.of(), null) + ); + } + + // --- helpers --- + + private DataFormat mockFormat(String name, long priority, Set fields) { + return new DataFormat() { + @Override + public String name() { + return name; + } + + @Override + public long priority() { + return priority; + } + + @Override + public Set supportedFields() { + return fields; + } + }; + } + + private MappedFieldType mockFieldType(String typeName) { + return new MappedFieldType(typeName, true, false, true, TextSearchInfo.NONE, Map.of()) { + @Override + public String typeName() { + return typeName; + } + + @Override + public ValueFetcher valueFetcher(QueryShardContext context, SearchLookup searchLookup, String format) { + return null; + } + + @Override + public Query termQuery(Object value, QueryShardContext context) { + return null; + } + }; + } + + /** + * Simple recording implementation of DocumentInput for test assertions. + */ + static class RecordingDocumentInput implements DocumentInput { + final List addedFields = new ArrayList<>(); + final List rowIds = new ArrayList<>(); + + @Override + public void addField(MappedFieldType fieldType, Object value) { + addedFields.add(value); + } + + @Override + public void setRowId(String rowIdFieldName, long rowId) { + rowIds.add(rowId); + } + + @Override + public Object getFinalInput() { + return null; + } + + @Override + public void close() {} + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java new file mode 100644 index 0000000000000..7fd3daf67031a --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java @@ -0,0 +1,211 @@ +/* + * 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.composite; + +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DataFormatPlugin; +import org.opensearch.plugins.ExtensiblePlugin; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Tests for {@link CompositeEnginePlugin}. + */ +public class CompositeEnginePluginTests extends OpenSearchTestCase { + + public void testGetSettingsReturnsAllThreeSettings() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + List> settings = plugin.getSettings(); + assertEquals(3, settings.size()); + assertTrue(settings.contains(CompositeEnginePlugin.COMPOSITE_ENABLED)); + assertTrue(settings.contains(CompositeEnginePlugin.PRIMARY_DATA_FORMAT)); + assertTrue(settings.contains(CompositeEnginePlugin.SECONDARY_DATA_FORMATS)); + } + + public void testCompositeEnabledDefaultsToFalse() { + Settings settings = Settings.builder().build(); + assertFalse(CompositeEnginePlugin.COMPOSITE_ENABLED.get(settings)); + } + + public void testPrimaryDataFormatDefaultsToLucene() { + Settings settings = Settings.builder().build(); + assertEquals("lucene", CompositeEnginePlugin.PRIMARY_DATA_FORMAT.get(settings)); + } + + public void testSecondaryDataFormatsDefaultsToEmpty() { + Settings settings = Settings.builder().build(); + assertTrue(CompositeEnginePlugin.SECONDARY_DATA_FORMATS.get(settings).isEmpty()); + } + + public void testCompositeEnabledValidatorRejectsEmptyPrimaryFormat() { + // Directly invoke the cross-setting validator with enabled=true and empty primary + Setting.Validator validator = extractValidator(); + Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, ""); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(true, deps)); + assertTrue(ex.getMessage().contains("index.composite.primary_data_format")); + } + + public void testCompositeEnabledValidatorAcceptsNonEmptyPrimaryFormat() { + Setting.Validator validator = extractValidator(); + Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, "lucene"); + // Should not throw + validator.validate(true, deps); + } + + public void testCompositeEnabledValidatorSkipsWhenDisabled() { + Setting.Validator validator = extractValidator(); + Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, ""); + // Should not throw when composite is disabled, even with empty primary + validator.validate(false, deps); + } + + @SuppressWarnings("unchecked") + private Setting.Validator extractValidator() { + // The COMPOSITE_ENABLED setting has a validator; extract it via the setting's properties + // We test the validator logic by constructing the dependency map directly + return new Setting.Validator<>() { + @Override + public void validate(Boolean value) {} + + @Override + public void validate(Boolean enabled, Map, Object> settings) { + if (enabled) { + String primary = (String) settings.get(CompositeEnginePlugin.PRIMARY_DATA_FORMAT); + if (primary == null || primary.isEmpty()) { + throw new IllegalArgumentException( + "[index.composite.enabled] requires [index.composite.primary_data_format] to be set" + ); + } + } + } + + @Override + public java.util.Iterator> settings() { + return List.>of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT).iterator(); + } + }; + } + + public void testGetDataFormatReturnsNull() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + assertNull(plugin.getDataFormat()); + } + + public void testLoadExtensionsRegistersPlugins() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + DataFormatPlugin lucenePlugin = CompositeTestHelper.stubPlugin("lucene", 1); + DataFormatPlugin parquetPlugin = CompositeTestHelper.stubPlugin("parquet", 2); + + plugin.loadExtensions(new ExtensiblePlugin.ExtensionLoader() { + @Override + @SuppressWarnings("unchecked") + public List loadExtensions(Class extensionPointType) { + if (extensionPointType == DataFormatPlugin.class) { + return (List) List.of(lucenePlugin, parquetPlugin); + } + return Collections.emptyList(); + } + }); + + Map plugins = plugin.getDataFormatPlugins(); + assertEquals(2, plugins.size()); + assertTrue(plugins.containsKey("lucene")); + assertTrue(plugins.containsKey("parquet")); + } + + public void testLoadExtensionsHigherPriorityWins() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + DataFormatPlugin lowPriority = CompositeTestHelper.stubPlugin("lucene", 1); + DataFormatPlugin highPriority = CompositeTestHelper.stubPlugin("lucene", 100); + + plugin.loadExtensions(new ExtensiblePlugin.ExtensionLoader() { + @Override + @SuppressWarnings("unchecked") + public List loadExtensions(Class extensionPointType) { + if (extensionPointType == DataFormatPlugin.class) { + return (List) List.of(lowPriority, highPriority); + } + return Collections.emptyList(); + } + }); + + Map plugins = plugin.getDataFormatPlugins(); + assertEquals(1, plugins.size()); + // The high priority one should win + assertEquals(100, plugins.get("lucene").getDataFormat().priority()); + } + + public void testLoadExtensionsSkipsNullDataFormat() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + DataFormatPlugin nullPlugin = new DataFormatPlugin() { + @Override + public DataFormat getDataFormat() { + return null; + } + + @Override + public org.opensearch.index.engine.dataformat.IndexingExecutionEngine indexingEngine( + org.opensearch.index.mapper.MapperService mapperService, + org.opensearch.index.shard.ShardPath shardPath, + IndexSettings indexSettings, + org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool writerPool + ) { + return null; + } + }; + + plugin.loadExtensions(new ExtensiblePlugin.ExtensionLoader() { + @Override + @SuppressWarnings("unchecked") + public List loadExtensions(Class extensionPointType) { + if (extensionPointType == DataFormatPlugin.class) { + return (List) List.of(nullPlugin); + } + return Collections.emptyList(); + } + }); + + assertTrue(plugin.getDataFormatPlugins().isEmpty()); + } + + public void testLoadExtensionsWithEmptyList() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + plugin.loadExtensions(new ExtensiblePlugin.ExtensionLoader() { + @Override + public List loadExtensions(Class extensionPointType) { + return Collections.emptyList(); + } + }); + + assertTrue(plugin.getDataFormatPlugins().isEmpty()); + } + + public void testGetDataFormatPluginsReturnsUnmodifiableMap() { + CompositeEnginePlugin plugin = new CompositeEnginePlugin(); + plugin.loadExtensions(new ExtensiblePlugin.ExtensionLoader() { + @Override + @SuppressWarnings("unchecked") + public List loadExtensions(Class extensionPointType) { + if (extensionPointType == DataFormatPlugin.class) { + return (List) List.of(CompositeTestHelper.stubPlugin("lucene", 1)); + } + return Collections.emptyList(); + } + }); + + Map plugins = plugin.getDataFormatPlugins(); + expectThrows(UnsupportedOperationException.class, () -> plugins.put("new", CompositeTestHelper.stubPlugin("new", 1))); + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java new file mode 100644 index 0000000000000..a91c4baeb3f50 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java @@ -0,0 +1,215 @@ +/* + * 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.composite; + +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.dataformat.DataFormatPlugin; +import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Tests for {@link CompositeIndexingExecutionEngine}. + */ +public class CompositeIndexingExecutionEngineTests extends OpenSearchTestCase { + + public void testConstructorWithPrimaryOnly() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene"); + assertNotNull(engine.getPrimaryDelegate()); + assertTrue(engine.getSecondaryDelegates().isEmpty()); + assertEquals("composite", engine.getDataFormat().name()); + } + + public void testConstructorWithPrimaryAndSecondary() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + assertNotNull(engine.getPrimaryDelegate()); + assertEquals(1, engine.getSecondaryDelegates().size()); + assertEquals("parquet", engine.getSecondaryDelegates().get(0).getDataFormat().name()); + } + + public void testConstructorWithMultipleSecondaries() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet", "arrow"); + assertEquals(2, engine.getSecondaryDelegates().size()); + } + + public void testConstructorThrowsWhenCompositeDisabled() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + Settings settings = Settings.builder() + .put("index.composite.enabled", false) + .put("index.composite.primary_data_format", "lucene") + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + + DataformatAwareLockableWriterPool pool = stubPool(); + expectThrows(IllegalStateException.class, () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool)); + } + + public void testConstructorThrowsWhenPrimaryFormatNotRegistered() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + IndexSettings indexSettings = createIndexSettings(true, "parquet"); + DataformatAwareLockableWriterPool pool = stubPool(); + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool) + ); + assertTrue(ex.getMessage().contains("parquet")); + } + + public void testConstructorThrowsWhenSecondaryFormatNotRegistered() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + Settings settings = Settings.builder() + .put("index.composite.enabled", true) + .put("index.composite.primary_data_format", "lucene") + .putList("index.composite.secondary_data_formats", "parquet") + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + + DataformatAwareLockableWriterPool pool = stubPool(); + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool) + ); + assertTrue(ex.getMessage().contains("parquet")); + } + + public void testConstructorRejectsNullDataFormatPlugins() { + IndexSettings indexSettings = createIndexSettings(true, "lucene"); + DataformatAwareLockableWriterPool pool = stubPool(); + expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(null, indexSettings, null, null, pool)); + } + + public void testConstructorRejectsNullIndexSettings() { + Map plugins = Map.of("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + DataformatAwareLockableWriterPool pool = stubPool(); + expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(plugins, null, null, null, pool)); + } + + public void testValidateFormatsRegisteredAcceptsValidConfig() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + plugins.put("parquet", CompositeTestHelper.stubPlugin("parquet", 2)); + + CompositeIndexingExecutionEngine.validateFormatsRegistered(plugins, "lucene", List.of("parquet")); + } + + public void testValidateFormatsRegisteredRejectsMissingPrimary() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> CompositeIndexingExecutionEngine.validateFormatsRegistered(plugins, "parquet", List.of()) + ); + assertTrue(ex.getMessage().contains("parquet")); + } + + public void testValidateFormatsRegisteredRejectsMissingSecondary() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> CompositeIndexingExecutionEngine.validateFormatsRegistered(plugins, "lucene", List.of("parquet")) + ); + assertTrue(ex.getMessage().contains("parquet")); + } + + public void testValidateFormatsRegisteredRejectsSecondaryEqualToPrimary() { + Map plugins = new HashMap<>(); + plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); + + IllegalStateException ex = expectThrows( + IllegalStateException.class, + () -> CompositeIndexingExecutionEngine.validateFormatsRegistered(plugins, "lucene", List.of("lucene")) + ); + assertTrue(ex.getMessage().contains("same as primary")); + } + + public void testCreateWriterReturnsCompositeWriter() throws IOException { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene"); + CompositeWriter writer = (CompositeWriter) engine.createWriter(42L); + assertNotNull(writer); + assertEquals(42L, writer.getWriterGeneration()); + writer.close(); + } + + public void testGetMergerDelegatesToPrimary() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene"); + assertNull(engine.getMerger()); + } + + public void testGetNativeBytesUsedSumsAllEngines() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + assertEquals(0L, engine.getNativeBytesUsed()); + } + + public void testGetDataFormatReturnsCompositeDataFormat() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + CompositeDataFormat format = engine.getDataFormat(); + assertNotNull(format); + assertEquals("composite", format.name()); + assertEquals(2, format.getDataFormats().size()); + } + + public void testNewDocumentInputReturnsCompositeDocumentInput() { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + CompositeDocumentInput input = engine.newDocumentInput(); + assertNotNull(input); + assertNotNull(input.getPrimaryInput()); + assertEquals(1, input.getSecondaryInputs().size()); + input.close(); + } + + public void testDeleteFilesDoesNotThrow() throws Exception { + CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + engine.deleteFiles(Map.of()); + } + + private IndexSettings createIndexSettings(boolean compositeEnabled, String primaryFormat) { + Settings settings = Settings.builder() + .put("index.composite.enabled", compositeEnabled) + .put("index.composite.primary_data_format", primaryFormat) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); + return new IndexSettings(indexMetadata, Settings.EMPTY); + } + + /** + * Creates an uninitialized pool for tests that only exercise constructor validation + * and expect exceptions before the pool is ever used. + */ + private static DataformatAwareLockableWriterPool stubPool() { + return new DataformatAwareLockableWriterPool<>(ConcurrentLinkedQueue::new, 1); + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java new file mode 100644 index 0000000000000..3e03b4fc0222c --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java @@ -0,0 +1,233 @@ +/* + * 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.composite; + +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.dataformat.DataFormat; +import org.opensearch.index.engine.dataformat.DataFormatPlugin; +import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; +import org.opensearch.index.engine.dataformat.DocumentInput; +import org.opensearch.index.engine.dataformat.FieldTypeCapabilities; +import org.opensearch.index.engine.dataformat.FileInfos; +import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; +import org.opensearch.index.engine.dataformat.Merger; +import org.opensearch.index.engine.dataformat.RefreshInput; +import org.opensearch.index.engine.dataformat.RefreshResult; +import org.opensearch.index.engine.dataformat.WriteResult; +import org.opensearch.index.engine.dataformat.Writer; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.shard.ShardPath; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Shared test utilities for composite engine tests. + */ +final class CompositeTestHelper { + + private CompositeTestHelper() {} + + /** + * Creates a CompositeIndexingExecutionEngine with stub engines for testing. + */ + static CompositeIndexingExecutionEngine createStubEngine(String primaryName, String... secondaryNames) { + Map plugins = new HashMap<>(); + plugins.put(primaryName, stubPlugin(primaryName, 1)); + for (String name : secondaryNames) { + plugins.put(name, stubPlugin(name, 2)); + } + + Settings.Builder settingsBuilder = Settings.builder() + .put("index.composite.enabled", true) + .put("index.composite.primary_data_format", primaryName) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1); + + if (secondaryNames.length > 0) { + settingsBuilder.putList("index.composite.secondary_data_formats", secondaryNames); + } + + Settings settings = settingsBuilder.build(); + IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + + DataformatAwareLockableWriterPool writerPool = new DataformatAwareLockableWriterPool<>( + ConcurrentLinkedQueue::new, + Runtime.getRuntime().availableProcessors() + ); + + return new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, writerPool); + } + + static DataFormatPlugin stubPlugin(String formatName, long priority) { + DataFormat format = stubFormat(formatName, priority, Set.of()); + return new DataFormatPlugin() { + @Override + public DataFormat getDataFormat() { + return format; + } + + @Override + public IndexingExecutionEngine indexingEngine( + MapperService mapperService, + ShardPath shardPath, + IndexSettings indexSettings, + @Nullable DataformatAwareLockableWriterPool writerPool + ) { + return new StubIndexingExecutionEngine(format); + } + }; + } + + static DataFormatPlugin stubPlugin(String formatName, long priority, Set fields) { + DataFormat format = stubFormat(formatName, priority, fields); + return new DataFormatPlugin() { + @Override + public DataFormat getDataFormat() { + return format; + } + + @Override + public IndexingExecutionEngine indexingEngine( + MapperService mapperService, + ShardPath shardPath, + IndexSettings indexSettings, + @Nullable DataformatAwareLockableWriterPool writerPool + ) { + return new StubIndexingExecutionEngine(format); + } + }; + } + + static DataFormat stubFormat(String name, long priority, Set fields) { + return new DataFormat() { + @Override + public String name() { + return name; + } + + @Override + public long priority() { + return priority; + } + + @Override + public Set supportedFields() { + return fields; + } + + @Override + public String toString() { + return "StubDataFormat{" + name + "}"; + } + }; + } + + /** + * Minimal stub IndexingExecutionEngine that returns no-op writers and empty results. + */ + static class StubIndexingExecutionEngine implements IndexingExecutionEngine> { + + private final DataFormat dataFormat; + + StubIndexingExecutionEngine(DataFormat dataFormat) { + this.dataFormat = dataFormat; + } + + @Override + public Writer> createWriter(long writerGeneration) { + return new StubWriter(dataFormat); + } + + @Override + public Merger getMerger() { + return null; + } + + @Override + public RefreshResult refresh(RefreshInput refreshInput) { + return new RefreshResult(Collections.emptyList()); + } + + @Override + public DataFormat getDataFormat() { + return dataFormat; + } + + @Override + public void deleteFiles(Map> filesToDelete) {} + + @Override + public DocumentInput newDocumentInput() { + return new StubDocumentInput(); + } + } + + /** + * Minimal stub Writer that always succeeds and returns empty FileInfos. + */ + static class StubWriter implements Writer> { + + private final DataFormat format; + private WriteResult resultToReturn = new WriteResult.Success(1, 1, 1); + + StubWriter(DataFormat format) { + this.format = format; + } + + void setResultToReturn(WriteResult result) { + this.resultToReturn = result; + } + + @Override + public WriteResult addDoc(DocumentInput d) { + return resultToReturn; + } + + @Override + public FileInfos flush() { + return FileInfos.empty(); + } + + @Override + public void sync() {} + + @Override + public void close() {} + } + + /** + * Minimal stub DocumentInput. + */ + static class StubDocumentInput implements DocumentInput { + @Override + public Object getFinalInput() { + return null; + } + + @Override + public void addField(org.opensearch.index.mapper.MappedFieldType fieldType, Object value) {} + + @Override + public void setRowId(String rowIdFieldName, long rowId) {} + + @Override + public void close() {} + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java new file mode 100644 index 0000000000000..c34f28fca0ca0 --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java @@ -0,0 +1,98 @@ +/* + * 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.composite; + +import org.opensearch.index.engine.dataformat.FileInfos; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +/** + * Tests for {@link CompositeWriter}. + */ +public class CompositeWriterTests extends OpenSearchTestCase { + + private CompositeIndexingExecutionEngine engine; + + @Override + public void setUp() throws Exception { + super.setUp(); + engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); + } + + public void testWriterGenerationIsPreserved() throws IOException { + long gen = randomLongBetween(0, 1000); + CompositeWriter writer = new CompositeWriter(engine, gen); + assertEquals(gen, writer.getWriterGeneration()); + writer.close(); + } + + public void testAbortedDefaultsToFalse() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + assertFalse(writer.isAborted()); + writer.close(); + } + + public void testAbortSetsAbortedFlag() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.abort(); + assertTrue(writer.isAborted()); + writer.close(); + } + + public void testFlushPendingDefaultsToFalse() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + assertFalse(writer.isFlushPending()); + writer.close(); + } + + public void testSetFlushPendingSetsFlag() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.setFlushPending(); + assertTrue(writer.isFlushPending()); + writer.close(); + } + + public void testLockAndUnlock() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.lock(); + assertTrue(writer.tryLock()); + writer.unlock(); + writer.unlock(); + writer.close(); + } + + public void testTryLockSucceedsWhenUnlocked() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + assertTrue(writer.tryLock()); + writer.unlock(); + writer.close(); + } + + public void testFlushReturnsFileInfos() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + FileInfos fileInfos = writer.flush(); + assertNotNull(fileInfos); + writer.close(); + } + + public void testSyncDoesNotThrow() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.sync(); + writer.close(); + } + + public void testCloseDoesNotThrow() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.close(); + // calling close again should also not throw + writer.close(); + } + +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java new file mode 100644 index 0000000000000..2dbe050d517be --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java @@ -0,0 +1,137 @@ +/* + * 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.composite; + +import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Tests for {@link DataformatAwareLockableWriterPool} with composite-specific callbacks. + */ +public class DataformatAwareLockableWriterPoolTests extends OpenSearchTestCase { + + private CompositeIndexingExecutionEngine engine; + + @Override + public void setUp() throws Exception { + super.setUp(); + engine = CompositeTestHelper.createStubEngine("lucene"); + } + + private DataformatAwareLockableWriterPool createPool(int concurrency) { + AtomicLong gen = new AtomicLong(0); + DataformatAwareLockableWriterPool pool = new DataformatAwareLockableWriterPool<>( + ConcurrentLinkedQueue::new, + concurrency + ); + pool.initialize(() -> new CompositeWriter(engine, gen.getAndIncrement())); + return pool; + } + + public void testGetAndLockReturnsNewWriterWhenPoolEmpty() { + DataformatAwareLockableWriterPool pool = createPool(2); + + CompositeWriter writer = pool.getAndLock(); + assertNotNull(writer); + assertEquals(0L, writer.getWriterGeneration()); + pool.releaseAndUnlock(writer); + } + + public void testReleaseAndUnlockMakesWriterReusable() { + DataformatAwareLockableWriterPool pool = createPool(2); + + CompositeWriter writer = pool.getAndLock(); + pool.releaseAndUnlock(writer); + + CompositeWriter reused = pool.getAndLock(); + assertSame(writer, reused); + pool.releaseAndUnlock(reused); + } + + public void testGetAndLockThrowsWhenClosed() throws IOException { + DataformatAwareLockableWriterPool pool = createPool(2); + pool.close(); + expectThrows(AlreadyClosedException.class, pool::getAndLock); + } + + public void testCheckoutAllReturnsRegisteredWriters() { + DataformatAwareLockableWriterPool pool = createPool(4); + + CompositeWriter w1 = pool.getAndLock(); + CompositeWriter w2 = pool.getAndLock(); + pool.releaseAndUnlock(w1); + pool.releaseAndUnlock(w2); + + List checkedOut = pool.checkoutAll(); + assertEquals(2, checkedOut.size()); + for (CompositeWriter w : checkedOut) { + assertFalse(w.isFlushPending()); + } + } + + public void testCheckoutAllReturnsEmptyWhenNoWriters() { + DataformatAwareLockableWriterPool pool = createPool(2); + + List checkedOut = pool.checkoutAll(); + assertTrue(checkedOut.isEmpty()); + } + + public void testCheckoutAllReturnsUnmodifiableList() { + DataformatAwareLockableWriterPool pool = createPool(2); + + CompositeWriter w = pool.getAndLock(); + pool.releaseAndUnlock(w); + + List checkedOut = pool.checkoutAll(); + expectThrows(UnsupportedOperationException.class, () -> checkedOut.add(new CompositeWriter(engine, 99))); + } + + public void testIsRegisteredReturnsTrueForPooledWriter() { + DataformatAwareLockableWriterPool pool = createPool(2); + + CompositeWriter writer = pool.getAndLock(); + assertTrue(pool.isRegistered(writer)); + pool.releaseAndUnlock(writer); + } + + public void testIsRegisteredReturnsFalseForUnknownWriter() { + DataformatAwareLockableWriterPool pool = createPool(2); + + CompositeWriter unknown = new CompositeWriter(engine, 99); + assertFalse(pool.isRegistered(unknown)); + } + + public void testCheckoutAllThrowsWhenClosed() throws IOException { + DataformatAwareLockableWriterPool pool = createPool(2); + pool.close(); + expectThrows(AlreadyClosedException.class, pool::checkoutAll); + } + + public void testIteratorReturnsSnapshotOfWriters() { + DataformatAwareLockableWriterPool pool = createPool(4); + + CompositeWriter w1 = pool.getAndLock(); + CompositeWriter w2 = pool.getAndLock(); + pool.releaseAndUnlock(w1); + pool.releaseAndUnlock(w2); + + int count = 0; + for (CompositeWriter w : pool) { + assertNotNull(w); + count++; + } + assertEquals(2, count); + } +} diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java new file mode 100644 index 0000000000000..1568be65a093c --- /dev/null +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java @@ -0,0 +1,53 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.composite; + +import org.opensearch.test.OpenSearchTestCase; + +/** + * Tests for {@link RowIdGenerator}. + */ +public class RowIdGeneratorTests extends OpenSearchTestCase { + + public void testNextRowIdStartsAtZero() { + RowIdGenerator generator = new RowIdGenerator("test"); + assertEquals(0L, generator.nextRowId()); + } + + public void testNextRowIdIncrementsMonotonically() { + RowIdGenerator generator = new RowIdGenerator("test"); + for (int i = 0; i < 100; i++) { + assertEquals(i, generator.nextRowId()); + } + } + + public void testCurrentRowIdReturnsCurrentWithoutIncrementing() { + RowIdGenerator generator = new RowIdGenerator("test"); + assertEquals(0L, generator.currentRowId()); + assertEquals(0L, generator.currentRowId()); + generator.nextRowId(); + assertEquals(1L, generator.currentRowId()); + assertEquals(1L, generator.currentRowId()); + } + + public void testGetSourceReturnsConstructorArgument() { + String source = randomAlphaOfLength(10); + RowIdGenerator generator = new RowIdGenerator(source); + assertEquals(source, generator.getSource()); + } + + public void testCurrentRowIdReflectsNextRowIdCalls() { + RowIdGenerator generator = new RowIdGenerator("test"); + int count = randomIntBetween(1, 50); + for (int i = 0; i < count; i++) { + generator.nextRowId(); + } + assertEquals(count, generator.currentRowId()); + } +} diff --git a/server/build.gradle b/server/build.gradle index 07010938f8975..2d36400327d73 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -65,6 +65,7 @@ sourceSets { dependencies { api project(':libs:opensearch-common') + api project(':libs:opensearch-concurrent-queue') api project(':libs:opensearch-core') api project(":libs:opensearch-compress") api project(':libs:opensearch-secure-sm') diff --git a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormat.java b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormat.java index 668571b97e55c..ed261b976403f 100644 --- a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormat.java +++ b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormat.java @@ -10,24 +10,26 @@ import org.opensearch.common.annotation.ExperimentalApi; +import java.util.Objects; import java.util.Set; /** * Represents a data format for storing and managing index data, with declared capabilities. * Each data format (e.g., Lucene, Parquet) declares what storage and query capabilities it supports. *

- * Equality is based on the format name — there should be one {@code DataFormat} instance per unique name. + * Equality is based on the format {@link #name()} — there should be one {@code DataFormat} instance + * per unique name. This allows {@code DataFormat} to be used safely as a {@link java.util.Map} key. * * @opensearch.experimental */ @ExperimentalApi -public interface DataFormat { +public abstract class DataFormat { /** * Returns the unique name of this data format. * * @return the data format name */ - String name(); + public abstract String name(); /** * Returns the priority of this data format. Higher priority formats are preferred @@ -35,12 +37,24 @@ public interface DataFormat { * * @return the priority value */ - long priority(); + public abstract long priority(); /** * Returns the set of field type capabilities supported by this data format. * * @return the supported field type capabilities */ - Set supportedFields(); + public abstract Set supportedFields(); + + @Override + public final boolean equals(Object o) { + if (this == o) return true; + if (o instanceof DataFormat == false) return false; + return Objects.equals(name(), ((DataFormat) o).name()); + } + + @Override + public final int hashCode() { + return Objects.hashCode(name()); + } } diff --git a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java index c2a1fc695652a..3ea6bdbd36fba 100644 --- a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java +++ b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java @@ -8,6 +8,7 @@ package org.opensearch.index.engine.dataformat; +import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; @@ -36,13 +37,14 @@ public interface DataFormatPlugin { * @param mapperService the mapper service for field mapping resolution * @param shardPath the shard path for file storage * @param indexSettings the index settings - * @param the data format type - * @param

the document input type + * @param writerPool the writer pool for managing writer instances, or {@code null} if the engine + * does not require external writer pool management (e.g., individual format engines) * @return the indexing execution engine instance */ - > IndexingExecutionEngine indexingEngine( + IndexingExecutionEngine indexingEngine( MapperService mapperService, ShardPath shardPath, - IndexSettings indexSettings + IndexSettings indexSettings, + @Nullable DataformatAwareLockableWriterPool writerPool ); } diff --git a/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java b/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java new file mode 100644 index 0000000000000..f29cf2345ab37 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java @@ -0,0 +1,166 @@ +/* + * 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.index.engine.dataformat; + +import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.common.SetOnce; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.queue.Lockable; +import org.opensearch.common.queue.LockableConcurrentQueue; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.function.Supplier; + +/** + * A data-format-aware pool of {@link Writer} instances backed by a {@link LockableConcurrentQueue}. + * Writers are locked on checkout and unlocked on release, ensuring thread-safe reuse. + *

+ * The pool is created with queue configuration only. The writer supplier is set later + * via {@link #initialize(Supplier)}, allowing the engine that owns the pool to provide + * the writer factory after construction (avoiding circular dependencies). + * + * @param the writer type, must implement both {@link Writer} and {@link Lockable} + * + * @opensearch.experimental + */ +@ExperimentalApi +public final class DataformatAwareLockableWriterPool & Lockable> implements Iterable, Closeable { + + private final Set writers; + private final LockableConcurrentQueue availableWriters; + private final SetOnce closed = new SetOnce<>(); + private volatile Supplier writerSupplier; + + /** + * Creates a new writer pool. The pool must be initialized + * with a writer supplier via {@link #initialize(Supplier)} before use. + * + * @param queueSupplier supplier for the underlying queue instances + * @param concurrency the concurrency level (number of stripes) + */ + public DataformatAwareLockableWriterPool(Supplier> queueSupplier, int concurrency) { + this.writers = Collections.newSetFromMap(new IdentityHashMap<>()); + this.availableWriters = new LockableConcurrentQueue<>(queueSupplier, concurrency); + } + + /** + * Initializes the pool with a writer supplier. Must be called exactly once before + * any checkout operations. + * + * @param writerSupplier factory for creating new writer instances + * @throws IllegalStateException if already initialized + */ + public void initialize(Supplier writerSupplier) { + if (this.writerSupplier != null) { + throw new IllegalStateException("DataformatAwareLockableWriterPool is already initialized"); + } + this.writerSupplier = Objects.requireNonNull(writerSupplier, "writerSupplier must not be null"); + } + + /** + * Locks and polls a writer from the pool, or creates a new one if none are available. + * + * @return a locked writer instance + */ + public W getAndLock() { + ensureOpen(); + ensureInitialized(); + W writer = availableWriters.lockAndPoll(); + return Objects.requireNonNullElseGet(writer, this::fetchWriter); + } + + private synchronized W fetchWriter() { + ensureOpen(); + W writer = writerSupplier.get(); + writer.lock(); + writers.add(writer); + return writer; + } + + /** + * Releases the given writer back to this pool for reuse. + * + * @param writer the writer to release + */ + public void releaseAndUnlock(W writer) { + assert isRegistered(writer) : "WriterPool doesn't know about this writer"; + availableWriters.addAndUnlock(writer); + } + + /** + * Lock and checkout all writers from the pool for flush. + * + * @return unmodifiable list of all writers locked by current thread + */ + public List checkoutAll() { + ensureOpen(); + List lockedWriters = new ArrayList<>(); + List checkedOutWriters = new ArrayList<>(); + for (W writer : this) { + writer.lock(); + lockedWriters.add(writer); + } + synchronized (this) { + for (W writer : lockedWriters) { + try { + if (isRegistered(writer) && writers.remove(writer)) { + availableWriters.remove(writer); + checkedOutWriters.add(writer); + } + } finally { + writer.unlock(); + } + } + } + return Collections.unmodifiableList(checkedOutWriters); + } + + /** + * Check if a writer is part of this pool. + * + * @param writer writer to validate + * @return true if the writer is part of this pool + */ + public synchronized boolean isRegistered(W writer) { + return writers.contains(writer); + } + + private void ensureOpen() { + if (closed.get() != null) { + throw new AlreadyClosedException("DataformatAwareLockableWriterPool is already closed"); + } + } + + private void ensureInitialized() { + if (writerSupplier == null) { + throw new IllegalStateException("DataformatAwareLockableWriterPool has not been initialized with a writer supplier"); + } + } + + @Override + public synchronized Iterator iterator() { + return List.copyOf(writers).iterator(); + } + + @Override + public void close() throws IOException { + if (closed.trySet(true) == false) { + throw new AlreadyClosedException("DataformatAwareLockableWriterPool is already closed"); + } + } +} diff --git a/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java b/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java index 0b967f718da97..99f0ea59f4067 100644 --- a/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java +++ b/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java @@ -65,11 +65,14 @@ public void testFullDataFormatLifecycle() throws IOException { .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), Version.CURRENT) .build(); - IndexingExecutionEngine engine = plugin.indexingEngine( - mock(MapperService.class), - new ShardPath(false, Path.of("/tmp/uuid/0"), Path.of("/tmp/uuid/0"), new ShardId("index", "uuid", 0)), - new IndexSettings(IndexMetadata.builder("index").settings(settings).build(), settings) - ); + @SuppressWarnings("unchecked") + IndexingExecutionEngine engine = (IndexingExecutionEngine) plugin + .indexingEngine( + mock(MapperService.class), + new ShardPath(false, Path.of("/tmp/uuid/0"), Path.of("/tmp/uuid/0"), new ShardId("index", "uuid", 0)), + new IndexSettings(IndexMetadata.builder("index").settings(settings).build(), settings), + null + ); assertEquals(format, engine.getDataFormat()); // 2. Create a writer and write documents @@ -225,7 +228,7 @@ public void testRefreshInput() { assertEquals(1, input.existingSegments().size()); } - static class MockDataFormat implements DataFormat { + static class MockDataFormat extends DataFormat { @Override public String name() { return "mock-columnar"; @@ -403,13 +406,13 @@ public DataFormat getDataFormat() { } @Override - @SuppressWarnings("unchecked") - public > IndexingExecutionEngine indexingEngine( + public IndexingExecutionEngine indexingEngine( MapperService mapperService, ShardPath shardPath, - IndexSettings indexSettings + IndexSettings indexSettings, + DataformatAwareLockableWriterPool writerPool ) { - return (IndexingExecutionEngine) new MockIndexingExecutionEngine(dataFormat); + return new MockIndexingExecutionEngine(dataFormat); } } From 7f98261d150a0e5c6fbbad8c095bba9f2e8f88f3 Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Mon, 23 Mar 2026 16:17:13 +0530 Subject: [PATCH 2/9] Address comments Signed-off-by: Arpit Bandejiya --- benchmarks/build.gradle | 1 + .../LockableConcurrentQueueBenchmark.java | 166 ++++++++++++++++++ .../common/queue/ConcurrentQueue.java | 25 +++ .../opensearch/common/queue/LockablePool.java | 136 ++++++++++++++ .../common/queue/LockablePoolTests.java | 146 +++++++++++++++ sandbox/libs/composite-common/build.gradle | 32 ++++ .../opensearch/composite/RowIdGenerator.java | 5 - .../opensearch/composite/package-info.java | 12 ++ .../composite/RowIdGeneratorTests.java | 0 sandbox/plugins/composite-engine/build.gradle | 1 + .../composite/CompositeDocumentInput.java | 20 ++- .../composite/CompositeEnginePlugin.java | 74 +------- .../CompositeIndexingExecutionEngine.java | 46 ++--- .../opensearch/composite/CompositeWriter.java | 55 ++++-- .../composite/CompositeEnginePluginTests.java | 62 +------ ...CompositeIndexingExecutionEngineTests.java | 50 +----- .../composite/CompositeTestHelper.java | 17 +- .../composite/CompositeWriterTests.java | 22 +++ ...ataformatAwareLockableWriterPoolTests.java | 137 --------------- .../engine/dataformat/DataFormatPlugin.java | 17 +- .../DataformatAwareLockableWriterPool.java | 166 ------------------ .../dataformat/DataFormatPluginTests.java | 10 +- 22 files changed, 654 insertions(+), 546 deletions(-) create mode 100644 benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java create mode 100644 libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java create mode 100644 libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockablePoolTests.java create mode 100644 sandbox/libs/composite-common/build.gradle rename sandbox/{plugins/composite-engine => libs/composite-common}/src/main/java/org/opensearch/composite/RowIdGenerator.java (93%) create mode 100644 sandbox/libs/composite-common/src/main/java/org/opensearch/composite/package-info.java rename sandbox/{plugins/composite-engine => libs/composite-common}/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java (100%) delete mode 100644 sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java delete mode 100644 server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java diff --git a/benchmarks/build.gradle b/benchmarks/build.gradle index 732e77934b427..21c8ed7a7741e 100644 --- a/benchmarks/build.gradle +++ b/benchmarks/build.gradle @@ -51,6 +51,7 @@ dependencies { // us to invoke the JMH uberjar as usual. exclude group: 'net.sf.jopt-simple', module: 'jopt-simple' } + api project(':libs:opensearch-concurrent-queue') api "org.openjdk.jmh:jmh-core:$versions.jmh" annotationProcessor "org.openjdk.jmh:jmh-generator-annprocess:$versions.jmh" // Dependencies of JMH diff --git a/benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java b/benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java new file mode 100644 index 0000000000000..cae90dae0b4d2 --- /dev/null +++ b/benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java @@ -0,0 +1,166 @@ +/* + * 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.benchmark.queue; + +import org.opensearch.common.queue.Lockable; +import org.opensearch.common.queue.LockableConcurrentQueue; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.LinkedList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; + +/** + * JMH benchmark for {@link LockableConcurrentQueue} measuring throughput of + * lock-and-poll / add-and-unlock cycles under varying concurrency levels. + *

+ * Includes two benchmark groups: + *

    + *
  • {@code pollAndReturn} — minimal overhead: poll an entry and immediately return it.
  • + *
  • {@code writerWorkload} — simulates a writer pool: poll an entry, perform simulated + * document writes (CPU work), then return the entry. Models the composite writer + * checkout-write-return cycle.
  • + *
+ */ +@Fork(3) +@Warmup(iterations = 5, time = 1) +@Measurement(iterations = 10, time = 1) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Benchmark) +@SuppressWarnings("unused") +public class LockableConcurrentQueueBenchmark { + + @Param({ "1", "4", "8" }) + int concurrency; + + @Param({ "16", "64" }) + int poolSize; + + private LockableConcurrentQueue queue; + + @Setup(Level.Iteration) + public void setup() { + queue = new LockableConcurrentQueue<>(LinkedList::new, concurrency); + for (int i = 0; i < poolSize; i++) { + LockableEntry entry = new LockableEntry(); + entry.lock(); + queue.addAndUnlock(entry); + } + } + + // ---- pollAndReturn: minimal overhead benchmarks ---- + + @Benchmark + @Threads(1) + public LockableEntry pollAndReturn_1thread() { + return pollAndReturn(); + } + + @Benchmark + @Threads(4) + public LockableEntry pollAndReturn_4threads() { + return pollAndReturn(); + } + + @Benchmark + @Threads(8) + public LockableEntry pollAndReturn_8threads() { + return pollAndReturn(); + } + + private LockableEntry pollAndReturn() { + LockableEntry entry = queue.lockAndPoll(); + if (entry != null) { + queue.addAndUnlock(entry); + } + return entry; + } + + // ---- writerWorkload: simulated writer pool benchmarks ---- + + @Benchmark + @Threads(4) + public void writerWorkload_4threads(Blackhole bh) { + writerWorkload(bh); + } + + @Benchmark + @Threads(8) + public void writerWorkload_8threads(Blackhole bh) { + writerWorkload(bh); + } + + @Benchmark + @Threads(16) + public void writerWorkload_16threads(Blackhole bh) { + writerWorkload(bh); + } + + /** + * Simulates a writer pool cycle: checkout an entry, perform CPU work + * representing document indexing across multiple formats, then return it. + */ + private void writerWorkload(Blackhole bh) { + LockableEntry entry = queue.lockAndPoll(); + if (entry != null) { + // Simulate document write work (field additions across formats) + bh.consume(simulateDocumentWrite(entry)); + queue.addAndUnlock(entry); + } + } + + /** + * Simulates the CPU cost of writing a document to multiple data formats. + * Performs arithmetic work to prevent JIT elimination while keeping + * the hold time realistic relative to a real addDoc call. + */ + private static long simulateDocumentWrite(LockableEntry entry) { + long result = entry.hashCode(); + // ~10 field additions across 2 formats + for (int i = 0; i < 20; i++) { + result ^= (result << 13); + result ^= (result >> 7); + result ^= (result << 17); + } + return result; + } + + static final class LockableEntry implements Lockable { + private final ReentrantLock lock = new ReentrantLock(); + + @Override + public void lock() { + lock.lock(); + } + + @Override + public boolean tryLock() { + return lock.tryLock(); + } + + @Override + public void unlock() { + lock.unlock(); + } + } +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java index f7856c9464f54..06fa238b4ad93 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java @@ -10,6 +10,7 @@ import java.util.Iterator; import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; @@ -32,6 +33,8 @@ public final class ConcurrentQueue { private final Lock[] locks; private final Queue[] queues; private final Supplier> queueSupplier; + /** Maps each entry to its queue index so that {@link #remove} can go directly to the right queue. */ + private final ConcurrentHashMap queueIndex; ConcurrentQueue(Supplier> queueSupplier, int concurrency) { if (concurrency < MIN_CONCURRENCY || concurrency > MAX_CONCURRENCY) { @@ -41,6 +44,7 @@ public final class ConcurrentQueue { } this.concurrency = concurrency; this.queueSupplier = queueSupplier; + this.queueIndex = new ConcurrentHashMap<>(); locks = new Lock[concurrency]; @SuppressWarnings({ "rawtypes", "unchecked" }) Queue[] queues = new Queue[concurrency]; @@ -63,6 +67,7 @@ void add(T entry) { if (lock.tryLock()) { try { queue.add(entry); + queueIndex.put(entry, index); return; } finally { lock.unlock(); @@ -75,6 +80,7 @@ void add(T entry) { lock.lock(); try { queue.add(entry); + queueIndex.put(entry, index); } finally { lock.unlock(); } @@ -93,6 +99,7 @@ T poll(Predicate predicate) { T entry = it.next(); if (predicate.test(entry)) { it.remove(); + queueIndex.remove(entry); return entry; } } @@ -112,6 +119,7 @@ T poll(Predicate predicate) { T entry = it.next(); if (predicate.test(entry)) { it.remove(); + queueIndex.remove(entry); return entry; } } @@ -123,12 +131,29 @@ T poll(Predicate predicate) { } boolean remove(T entry) { + Integer queueIdx = queueIndex.get(entry); + if (queueIdx != null) { + final Lock lock = locks[queueIdx]; + final Queue queue = queues[queueIdx]; + lock.lock(); + try { + if (queue.remove(entry)) { + queueIndex.remove(entry); + return true; + } + } finally { + lock.unlock(); + } + } + // Fallback: entry may have been re-added to a different queue between the index lookup + // and the lock acquisition, so scan all stripes. for (int i = 0; i < concurrency; ++i) { final Lock lock = locks[i]; final Queue queue = queues[i]; lock.lock(); try { if (queue.remove(entry)) { + queueIndex.remove(entry); return true; } } finally { diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java new file mode 100644 index 0000000000000..505c92abd5833 --- /dev/null +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java @@ -0,0 +1,136 @@ +/* + * 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.common.queue; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.function.Supplier; + +/** + * A thread-safe pool of {@link Lockable} items backed by a {@link LockableConcurrentQueue}. + * Items are locked on checkout and unlocked on release, ensuring safe reuse across threads. + *

+ * The pool is created with a supplier that produces new items on demand when the pool + * is empty. Items are tracked in a set for registration checks and iteration. + * + * @param the pooled item type, must implement {@link Lockable} + */ +public final class LockablePool implements Iterable, Closeable { + + private final Set items; + private final LockableConcurrentQueue availableItems; + private final Supplier itemSupplier; + private volatile boolean closed; + + /** + * Creates a new pool. + * + * @param itemSupplier factory for creating new items when the pool is empty + * @param queueSupplier supplier for the underlying queue instances + * @param concurrency the concurrency level (number of stripes) + */ + public LockablePool(Supplier itemSupplier, Supplier> queueSupplier, int concurrency) { + this.items = Collections.newSetFromMap(new IdentityHashMap<>()); + this.itemSupplier = Objects.requireNonNull(itemSupplier, "itemSupplier must not be null"); + this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); + } + + /** + * Locks and polls an item from the pool, or creates a new one if none are available. + * + * @return a locked item + * @throws IllegalStateException if the pool is closed + */ + public T getAndLock() { + ensureOpen(); + T item = availableItems.lockAndPoll(); + return Objects.requireNonNullElseGet(item, this::fetchItem); + } + + private synchronized T fetchItem() { + ensureOpen(); + T item = itemSupplier.get(); + item.lock(); + items.add(item); + return item; + } + + /** + * Releases the given item back to this pool for reuse. + * + * @param item the item to release + */ + public void releaseAndUnlock(T item) { + assert isRegistered(item) : "Pool doesn't know about this item"; + availableItems.addAndUnlock(item); + } + + /** + * Lock and checkout all items from the pool. + * + * @return unmodifiable list of all items locked by current thread + * @throws IllegalStateException if the pool is closed + */ + public List checkoutAll() { + ensureOpen(); + List lockedItems = new ArrayList<>(); + List checkedOutItems = new ArrayList<>(); + for (T item : this) { + item.lock(); + lockedItems.add(item); + } + synchronized (this) { + for (T item : lockedItems) { + try { + if (isRegistered(item) && items.remove(item)) { + availableItems.remove(item); + checkedOutItems.add(item); + } + } finally { + item.unlock(); + } + } + } + return Collections.unmodifiableList(checkedOutItems); + } + + /** + * Check if an item is part of this pool. + * + * @param item item to validate + * @return true if the item is part of this pool + */ + public synchronized boolean isRegistered(T item) { + return items.contains(item); + } + + private void ensureOpen() { + if (closed) { + throw new IllegalStateException("LockablePool is already closed"); + } + } + + @Override + public synchronized Iterator iterator() { + return List.copyOf(items).iterator(); + } + + @Override + public void close() throws IOException { + this.closed = true; + } +} diff --git a/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockablePoolTests.java b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockablePoolTests.java new file mode 100644 index 0000000000000..abe9218646faa --- /dev/null +++ b/libs/concurrent-queue/src/test/java/org/opensearch/common/queue/LockablePoolTests.java @@ -0,0 +1,146 @@ +/* + * 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.common.queue; + +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Tests for {@link LockablePool}. + */ +public class LockablePoolTests extends OpenSearchTestCase { + + /** + * A simple lockable entry for testing. + */ + static class LockableEntry implements Lockable { + final String id; + private final ReentrantLock delegate = new ReentrantLock(); + + LockableEntry(String id) { + this.id = id; + } + + @Override + public void lock() { + delegate.lock(); + } + + @Override + public boolean tryLock() { + return delegate.tryLock(); + } + + @Override + public void unlock() { + delegate.unlock(); + } + + boolean isHeldByCurrentThread() { + return delegate.isHeldByCurrentThread(); + } + } + + private static int idCounter = 0; + + private LockablePool createPool() { + return new LockablePool<>(() -> new LockableEntry("entry-" + idCounter++), LinkedList::new, 1); + } + + public void testGetAndLockReturnsLockedItem() { + LockablePool pool = createPool(); + LockableEntry item = pool.getAndLock(); + assertNotNull(item); + assertTrue(item.isHeldByCurrentThread()); + pool.releaseAndUnlock(item); + } + + public void testReleaseAndUnlockAllowsReuse() { + LockablePool pool = createPool(); + LockableEntry first = pool.getAndLock(); + pool.releaseAndUnlock(first); + + LockableEntry second = pool.getAndLock(); + assertSame("Released item should be reused", first, second); + pool.releaseAndUnlock(second); + } + + public void testClosedPoolThrowsOnGetAndLock() throws IOException { + LockablePool pool = createPool(); + pool.close(); + + IllegalStateException ex = expectThrows(IllegalStateException.class, pool::getAndLock); + assertEquals("LockablePool is already closed", ex.getMessage()); + } + + public void testCheckoutAllReturnsAllItems() { + LockablePool pool = createPool(); + LockableEntry a = pool.getAndLock(); + pool.releaseAndUnlock(a); + LockableEntry b = pool.getAndLock(); + pool.releaseAndUnlock(b); + + // a and b may be the same item (reuse), so pool may have 1 or 2 items + List all = pool.checkoutAll(); + assertFalse("checkoutAll should return at least one item", all.isEmpty()); + } + + public void testCheckoutAllOnEmptyPoolReturnsEmptyList() { + LockablePool pool = createPool(); + List all = pool.checkoutAll(); + assertTrue("checkoutAll on empty pool should return empty list", all.isEmpty()); + } + + public void testCheckoutAllReturnsUnmodifiableList() { + LockablePool pool = createPool(); + LockableEntry item = pool.getAndLock(); + pool.releaseAndUnlock(item); + + List all = pool.checkoutAll(); + expectThrows(UnsupportedOperationException.class, () -> all.add(new LockableEntry("rogue"))); + } + + public void testIsRegisteredTrueForPoolItem() { + LockablePool pool = createPool(); + LockableEntry item = pool.getAndLock(); + assertTrue(pool.isRegistered(item)); + pool.releaseAndUnlock(item); + } + + public void testIsRegisteredFalseForUnknownItem() { + LockablePool pool = createPool(); + LockableEntry stranger = new LockableEntry("stranger"); + assertFalse(pool.isRegistered(stranger)); + } + + public void testIteratorReturnsSnapshot() { + LockablePool pool = createPool(); + LockableEntry a = pool.getAndLock(); + pool.releaseAndUnlock(a); + + int count = 0; + for (LockableEntry item : pool) { + assertNotNull(item); + count++; + } + assertEquals(1, count); + } + + public void testClosedPoolThrowsOnCheckoutAll() throws IOException { + LockablePool pool = createPool(); + pool.close(); + + IllegalStateException ex = expectThrows(IllegalStateException.class, pool::checkoutAll); + assertEquals("LockablePool is already closed", ex.getMessage()); + } +} diff --git a/sandbox/libs/composite-common/build.gradle b/sandbox/libs/composite-common/build.gradle new file mode 100644 index 0000000000000..3400787defe92 --- /dev/null +++ b/sandbox/libs/composite-common/build.gradle @@ -0,0 +1,32 @@ +/* + * 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. + */ + +/* + * Shared utilities for the composite indexing engine. + * Pure Java — no external runtime dependencies. + */ + +dependencies { + /******* + * !!!! NO RUNTIME DEPENDENCIES !!!! + *******/ + + testImplementation "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" + testImplementation "junit:junit:${versions.junit}" + testImplementation "org.hamcrest:hamcrest:${versions.hamcrest}" + + testImplementation(project(":test:framework")) { + exclude group: 'org.opensearch', module: 'opensearch-composite-common' + } +} + +testingConventions.enabled = true + +tasks.named('forbiddenApisMain').configure { + replaceSignatureFiles 'jdk-signatures' +} diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java b/sandbox/libs/composite-common/src/main/java/org/opensearch/composite/RowIdGenerator.java similarity index 93% rename from sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java rename to sandbox/libs/composite-common/src/main/java/org/opensearch/composite/RowIdGenerator.java index 9f6c223ddb5db..1463e8c2890da 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/RowIdGenerator.java +++ b/sandbox/libs/composite-common/src/main/java/org/opensearch/composite/RowIdGenerator.java @@ -8,18 +8,13 @@ package org.opensearch.composite; -import org.opensearch.common.annotation.ExperimentalApi; - import java.util.concurrent.atomic.AtomicLong; /** * Generates monotonically increasing row IDs for cross-format document synchronization. * Each writer instance gets its own {@code RowIdGenerator} so that row IDs are unique * within a writer's segment scope. - * - * @opensearch.experimental */ -@ExperimentalApi public class RowIdGenerator { private final String source; diff --git a/sandbox/libs/composite-common/src/main/java/org/opensearch/composite/package-info.java b/sandbox/libs/composite-common/src/main/java/org/opensearch/composite/package-info.java new file mode 100644 index 0000000000000..0197370226c89 --- /dev/null +++ b/sandbox/libs/composite-common/src/main/java/org/opensearch/composite/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. + */ + +/** + * Common utilities shared across composite engine components. + */ +package org.opensearch.composite; diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java b/sandbox/libs/composite-common/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java similarity index 100% rename from sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java rename to sandbox/libs/composite-common/src/test/java/org/opensearch/composite/RowIdGeneratorTests.java diff --git a/sandbox/plugins/composite-engine/build.gradle b/sandbox/plugins/composite-engine/build.gradle index 527e2a904fe52..1547c0c33f445 100644 --- a/sandbox/plugins/composite-engine/build.gradle +++ b/sandbox/plugins/composite-engine/build.gradle @@ -13,6 +13,7 @@ opensearchplugin { dependencies { api project(':libs:opensearch-concurrent-queue') + api project(':sandbox:libs:composite-common') compileOnly project(':server') testImplementation project(':test:framework') } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java index ff7ec0593249d..d16e909dc64f1 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java @@ -58,9 +58,23 @@ public CompositeDocumentInput( @Override public void addField(MappedFieldType fieldType, Object value) { - primaryDocumentInput.addField(fieldType, value); - for (DocumentInput input : secondaryDocumentInputs.values()) { - input.addField(fieldType, value); + try { + primaryDocumentInput.addField(fieldType, value); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to add field [" + fieldType.name() + "] in primary format [" + primaryFormat.name() + "]", + e + ); + } + for (Map.Entry> entry : secondaryDocumentInputs.entrySet()) { + try { + entry.getValue().addField(fieldType, value); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to add field [" + fieldType.name() + "] in secondary format [" + entry.getKey().name() + "]", + e + ); + } } } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java index 2d86c07f96cd4..c87bf339beb0d 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java @@ -10,15 +10,11 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.settings.Setting; -import org.opensearch.common.settings.Settings; -import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.dataformat.DataFormat; import org.opensearch.index.engine.dataformat.DataFormatPlugin; -import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardPath; @@ -27,10 +23,8 @@ import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; /** * Sandbox plugin that provides a {@link CompositeIndexingExecutionEngine} for @@ -40,8 +34,8 @@ *

* Registers two index settings: *

    - *
  • {@code index.composite.enabled} — activates composite indexing (default {@code false})
  • *
  • {@code index.composite.primary_data_format} — designates the primary format (default {@code "lucene"})
  • + *
  • {@code index.composite.secondary_data_formats} — lists the secondary formats (default empty)
  • *
*

* Format plugins (e.g., Parquet) extend this plugin by declaring @@ -56,38 +50,6 @@ public class CompositeEnginePlugin extends Plugin implements ExtensiblePlugin, D private static final Logger logger = LogManager.getLogger(CompositeEnginePlugin.class); - /** - * Index setting to enable composite indexing for an index. - * When {@code true}, the composite engine orchestrates writes across all registered data formats. - * Validates that the primary data format is non-empty when enabled. - */ - public static final Setting COMPOSITE_ENABLED = Setting.boolSetting( - "index.composite.enabled", - false, - new Setting.Validator<>() { - @Override - public void validate(Boolean value) {} - - @Override - public void validate(Boolean enabled, Map, Object> settings) { - if (enabled) { - String primary = (String) settings.get(PRIMARY_DATA_FORMAT); - if (primary == null || primary.isEmpty()) { - throw new IllegalArgumentException( - "[index.composite.enabled] requires [index.composite.primary_data_format] to be set" - ); - } - } - } - - @Override - public Iterator> settings() { - return List.>of(PRIMARY_DATA_FORMAT).iterator(); - } - }, - Setting.Property.IndexScope - ); - /** * Index setting that designates the primary data format for an index. * The primary format is the authoritative format used for merge operations. @@ -95,7 +57,8 @@ public Iterator> settings() { public static final Setting PRIMARY_DATA_FORMAT = Setting.simpleString( "index.composite.primary_data_format", "lucene", - Setting.Property.IndexScope + Setting.Property.IndexScope, + Setting.Property.Final ); /** @@ -107,7 +70,8 @@ public Iterator> settings() { "index.composite.secondary_data_formats", Collections.emptyList(), s -> s, - Setting.Property.IndexScope + Setting.Property.IndexScope, + Setting.Property.Final ); /** @@ -163,20 +127,7 @@ public void loadExtensions(ExtensionLoader loader) { @Override public List> getSettings() { - return List.of(COMPOSITE_ENABLED, PRIMARY_DATA_FORMAT, SECONDARY_DATA_FORMATS); - } - - @Override - public void onIndexModule(IndexModule indexModule) { - Settings settings = indexModule.getSettings(); - boolean compositeEnabled = COMPOSITE_ENABLED.get(settings); - if (compositeEnabled == false) { - return; - } - - String primaryFormatName = PRIMARY_DATA_FORMAT.get(settings); - List secondaryFormatNames = SECONDARY_DATA_FORMATS.get(settings); - CompositeIndexingExecutionEngine.validateFormatsRegistered(dataFormatPlugins, primaryFormatName, secondaryFormatNames); + return List.of(PRIMARY_DATA_FORMAT, SECONDARY_DATA_FORMATS); } @Override @@ -186,17 +137,8 @@ public DataFormat getDataFormat() { } @Override - public IndexingExecutionEngine indexingEngine( - MapperService mapperService, - ShardPath shardPath, - IndexSettings indexSettings, - @Nullable DataformatAwareLockableWriterPool writerPool - ) { - Objects.requireNonNull(writerPool, "DataformatAwareLockableWriterPool is required for CompositeIndexingExecutionEngine"); - @SuppressWarnings("unchecked") - DataformatAwareLockableWriterPool compositeWriterPool = (DataformatAwareLockableWriterPool< - CompositeWriter>) writerPool; - return new CompositeIndexingExecutionEngine(dataFormatPlugins, indexSettings, mapperService, shardPath, compositeWriterPool); + public IndexingExecutionEngine indexingEngine(MapperService mapperService, ShardPath shardPath, IndexSettings indexSettings) { + return new CompositeIndexingExecutionEngine(dataFormatPlugins, indexSettings, mapperService, shardPath); } /** diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java index bbfbb0e5c3531..42c6f4e5a702e 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java @@ -11,11 +11,11 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.queue.LockablePool; import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.dataformat.DataFormat; import org.opensearch.index.engine.dataformat.DataFormatPlugin; -import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; import org.opensearch.index.engine.dataformat.DocumentInput; import org.opensearch.index.engine.dataformat.FileInfos; import org.opensearch.index.engine.dataformat.IndexingExecutionEngine; @@ -32,9 +32,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.LinkedHashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; /** @@ -56,9 +58,9 @@ public class CompositeIndexingExecutionEngine implements IndexingExecutionEngine private static final Logger logger = LogManager.getLogger(CompositeIndexingExecutionEngine.class); private final IndexingExecutionEngine primaryEngine; - private final List> secondaryEngines; + private final Set> secondaryEngines; private final CompositeDataFormat compositeDataFormat; - private final DataformatAwareLockableWriterPool writerPool; + private final LockablePool writerPool; /** * Constructs a CompositeIndexingExecutionEngine by reading index settings to @@ -70,35 +72,25 @@ public class CompositeIndexingExecutionEngine implements IndexingExecutionEngine * commit coordination. Secondary engines receive writes alongside the primary but * are not used as the merge authority. *

- * The pool is initialized with a writer supplier that creates {@link CompositeWriter} - * instances bound to this engine, eliminating the circular dependency between the - * engine and its writer pool. + * The writer pool is created internally and initialized with a writer supplier + * that creates {@link CompositeWriter} instances bound to this engine. * * @param dataFormatPlugins the discovered data format plugins keyed by format name * @param indexSettings the index settings containing composite configuration * @param mapperService the mapper service for field mapping resolution * @param shardPath the shard path for file storage - * @param writerPool the writer pool for managing composite writer instances - * @throws IllegalStateException if composite indexing is not enabled * @throws IllegalArgumentException if any configured format is not registered - * @throws NullPointerException if writerPool is null */ public CompositeIndexingExecutionEngine( Map dataFormatPlugins, IndexSettings indexSettings, MapperService mapperService, - ShardPath shardPath, - DataformatAwareLockableWriterPool writerPool + ShardPath shardPath ) { Objects.requireNonNull(dataFormatPlugins, "dataFormatPlugins must not be null"); Objects.requireNonNull(indexSettings, "indexSettings must not be null"); - Objects.requireNonNull(writerPool, "writerPool must not be null"); Settings settings = indexSettings.getSettings(); - boolean compositeEnabled = CompositeEnginePlugin.COMPOSITE_ENABLED.get(settings); - if (compositeEnabled == false) { - throw new IllegalStateException("Composite indexing is not enabled for index [" + indexSettings.getIndex().getName() + "]"); - } String primaryFormatName = CompositeEnginePlugin.PRIMARY_DATA_FORMAT.get(settings); List secondaryFormatNames = CompositeEnginePlugin.SECONDARY_DATA_FORMATS.get(settings); @@ -107,23 +99,26 @@ public CompositeIndexingExecutionEngine( List allFormats = new ArrayList<>(); DataFormatPlugin primaryPlugin = dataFormatPlugins.get(primaryFormatName); - this.primaryEngine = primaryPlugin.indexingEngine(mapperService, shardPath, indexSettings, null); + this.primaryEngine = primaryPlugin.indexingEngine(mapperService, shardPath, indexSettings); allFormats.add(primaryPlugin.getDataFormat()); List> secondaries = new ArrayList<>(); for (String secondaryName : secondaryFormatNames) { DataFormatPlugin secondaryPlugin = dataFormatPlugins.get(secondaryName); - secondaries.add(secondaryPlugin.indexingEngine(mapperService, shardPath, indexSettings, null)); + secondaries.add(secondaryPlugin.indexingEngine(mapperService, shardPath, indexSettings)); allFormats.add(secondaryPlugin.getDataFormat()); } - this.secondaryEngines = List.copyOf(secondaries); + this.secondaryEngines = Set.copyOf(secondaries); this.compositeDataFormat = new CompositeDataFormat(allFormats); - this.writerPool = writerPool; - // Initialize the pool with a writer supplier now that the engine is fully constructed + // Create the writer pool internally, matching the reference code pattern AtomicLong writerGenerationCounter = new AtomicLong(0); - writerPool.initialize(() -> new CompositeWriter(this, writerGenerationCounter.getAndIncrement())); + this.writerPool = new LockablePool<>( + () -> new CompositeWriter(this, writerGenerationCounter.getAndIncrement()), + LinkedList::new, + Runtime.getRuntime().availableProcessors() + ); } /** @@ -262,10 +257,7 @@ public CompositeDocumentInput newDocumentInput() { secondaryInputMap.put(engine.getDataFormat(), engine.newDocumentInput()); } return new CompositeDocumentInput(primaryEngine.getDataFormat(), primaryInput, secondaryInputMap, () -> { - assert writer.isFlushPending() == false && writer.isAborted() == false : "CompositeWriter has pending flush: " - + writer.isFlushPending() - + " aborted=" - + writer.isAborted(); + assert writer.getState() == CompositeWriter.WriterState.ACTIVE : "CompositeWriter is not ACTIVE, state=" + writer.getState(); writerPool.releaseAndUnlock(writer); }); } @@ -284,7 +276,7 @@ public CompositeDocumentInput newDocumentInput() { * * @return the secondary engines */ - public List> getSecondaryDelegates() { + public Set> getSecondaryDelegates() { return secondaryEngines; } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java index fe1338d4df020..d36bbaae8a3cc 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java @@ -25,6 +25,7 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; /** @@ -48,8 +49,26 @@ public class CompositeWriter implements Writer, Lockable private final ReentrantLock lock; private final long writerGeneration; private final RowIdGenerator rowIdGenerator; - private volatile boolean aborted; - private volatile boolean flushPending; + private final AtomicReference state; + + /** + * Represents the lifecycle state of a {@link CompositeWriter}. + *

+ * State transitions are one-way from {@code ACTIVE}: + *

    + *
  • {@code ACTIVE} → {@code FLUSH_PENDING} (when refresh marks the writer for flushing)
  • + *
  • {@code ACTIVE} → {@code ABORTED} (when the writer is aborted due to failure)
  • + *
+ */ + @ExperimentalApi + public enum WriterState { + /** Writer is actively accepting documents. */ + ACTIVE, + /** Writer has been marked for flushing and should not accept new documents. */ + FLUSH_PENDING, + /** Writer has been aborted due to a failure and should not be reused. */ + ABORTED + } /** * Constructs a CompositeWriter from the given engine and writer generation. @@ -64,8 +83,7 @@ public class CompositeWriter implements Writer, Lockable @SuppressWarnings("unchecked") public CompositeWriter(CompositeIndexingExecutionEngine engine, long writerGeneration) { this.lock = new ReentrantLock(); - this.aborted = false; - this.flushPending = false; + this.state = new AtomicReference<>(WriterState.ACTIVE); this.writerGeneration = writerGeneration; IndexingExecutionEngine primaryDelegate = engine.getPrimaryDelegate(); @@ -159,10 +177,14 @@ public long getWriterGeneration() { } /** - * Marks this writer as aborted. + * Marks this writer as aborted. Only transitions from {@code ACTIVE}. + * + * @throws IllegalStateException if the writer is not in {@code ACTIVE} state */ public void abort() { - this.aborted = true; + if (this.state.compareAndSet(WriterState.ACTIVE, WriterState.ABORTED) == false) { + throw new IllegalStateException("Cannot abort writer in state " + state.get()); + } } /** @@ -171,14 +193,18 @@ public void abort() { * @return {@code true} if aborted */ public boolean isAborted() { - return aborted; + return getState() == WriterState.ABORTED; } /** - * Marks this writer as having a pending flush. + * Marks this writer as having a pending flush. Only transitions from {@code ACTIVE}. + * + * @throws IllegalStateException if the writer is not in {@code ACTIVE} state */ public void setFlushPending() { - this.flushPending = true; + if (this.state.compareAndSet(WriterState.ACTIVE, WriterState.FLUSH_PENDING) == false) { + throw new IllegalStateException("Cannot set flush pending on writer in state " + state.get()); + } } /** @@ -187,7 +213,16 @@ public void setFlushPending() { * @return {@code true} if a flush is pending */ public boolean isFlushPending() { - return flushPending; + return getState() == WriterState.FLUSH_PENDING; + } + + /** + * Returns the current state of this writer. + * + * @return the writer state + */ + public WriterState getState() { + return state.get(); } @Override diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java index 7fd3daf67031a..c43fc891e3279 100644 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeEnginePluginTests.java @@ -25,20 +25,14 @@ */ public class CompositeEnginePluginTests extends OpenSearchTestCase { - public void testGetSettingsReturnsAllThreeSettings() { + public void testGetSettingsReturnsBothSettings() { CompositeEnginePlugin plugin = new CompositeEnginePlugin(); List> settings = plugin.getSettings(); - assertEquals(3, settings.size()); - assertTrue(settings.contains(CompositeEnginePlugin.COMPOSITE_ENABLED)); + assertEquals(2, settings.size()); assertTrue(settings.contains(CompositeEnginePlugin.PRIMARY_DATA_FORMAT)); assertTrue(settings.contains(CompositeEnginePlugin.SECONDARY_DATA_FORMATS)); } - public void testCompositeEnabledDefaultsToFalse() { - Settings settings = Settings.builder().build(); - assertFalse(CompositeEnginePlugin.COMPOSITE_ENABLED.get(settings)); - } - public void testPrimaryDataFormatDefaultsToLucene() { Settings settings = Settings.builder().build(); assertEquals("lucene", CompositeEnginePlugin.PRIMARY_DATA_FORMAT.get(settings)); @@ -49,55 +43,6 @@ public void testSecondaryDataFormatsDefaultsToEmpty() { assertTrue(CompositeEnginePlugin.SECONDARY_DATA_FORMATS.get(settings).isEmpty()); } - public void testCompositeEnabledValidatorRejectsEmptyPrimaryFormat() { - // Directly invoke the cross-setting validator with enabled=true and empty primary - Setting.Validator validator = extractValidator(); - Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, ""); - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> validator.validate(true, deps)); - assertTrue(ex.getMessage().contains("index.composite.primary_data_format")); - } - - public void testCompositeEnabledValidatorAcceptsNonEmptyPrimaryFormat() { - Setting.Validator validator = extractValidator(); - Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, "lucene"); - // Should not throw - validator.validate(true, deps); - } - - public void testCompositeEnabledValidatorSkipsWhenDisabled() { - Setting.Validator validator = extractValidator(); - Map, Object> deps = Map.of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT, ""); - // Should not throw when composite is disabled, even with empty primary - validator.validate(false, deps); - } - - @SuppressWarnings("unchecked") - private Setting.Validator extractValidator() { - // The COMPOSITE_ENABLED setting has a validator; extract it via the setting's properties - // We test the validator logic by constructing the dependency map directly - return new Setting.Validator<>() { - @Override - public void validate(Boolean value) {} - - @Override - public void validate(Boolean enabled, Map, Object> settings) { - if (enabled) { - String primary = (String) settings.get(CompositeEnginePlugin.PRIMARY_DATA_FORMAT); - if (primary == null || primary.isEmpty()) { - throw new IllegalArgumentException( - "[index.composite.enabled] requires [index.composite.primary_data_format] to be set" - ); - } - } - } - - @Override - public java.util.Iterator> settings() { - return List.>of(CompositeEnginePlugin.PRIMARY_DATA_FORMAT).iterator(); - } - }; - } - public void testGetDataFormatReturnsNull() { CompositeEnginePlugin plugin = new CompositeEnginePlugin(); assertNull(plugin.getDataFormat()); @@ -159,8 +104,7 @@ public DataFormat getDataFormat() { public org.opensearch.index.engine.dataformat.IndexingExecutionEngine indexingEngine( org.opensearch.index.mapper.MapperService mapperService, org.opensearch.index.shard.ShardPath shardPath, - IndexSettings indexSettings, - org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool writerPool + IndexSettings indexSettings ) { return null; } diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java index a91c4baeb3f50..fb6bc8f08c0f2 100644 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeIndexingExecutionEngineTests.java @@ -13,14 +13,12 @@ import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.dataformat.DataFormatPlugin; -import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; /** * Tests for {@link CompositeIndexingExecutionEngine}. @@ -38,7 +36,7 @@ public void testConstructorWithPrimaryAndSecondary() { CompositeIndexingExecutionEngine engine = CompositeTestHelper.createStubEngine("lucene", "parquet"); assertNotNull(engine.getPrimaryDelegate()); assertEquals(1, engine.getSecondaryDelegates().size()); - assertEquals("parquet", engine.getSecondaryDelegates().get(0).getDataFormat().name()); + assertEquals("parquet", engine.getSecondaryDelegates().iterator().next().getDataFormat().name()); } public void testConstructorWithMultipleSecondaries() { @@ -46,33 +44,14 @@ public void testConstructorWithMultipleSecondaries() { assertEquals(2, engine.getSecondaryDelegates().size()); } - public void testConstructorThrowsWhenCompositeDisabled() { - Map plugins = new HashMap<>(); - plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); - - Settings settings = Settings.builder() - .put("index.composite.enabled", false) - .put("index.composite.primary_data_format", "lucene") - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .build(); - IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); - IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); - - DataformatAwareLockableWriterPool pool = stubPool(); - expectThrows(IllegalStateException.class, () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool)); - } - public void testConstructorThrowsWhenPrimaryFormatNotRegistered() { Map plugins = new HashMap<>(); plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); - IndexSettings indexSettings = createIndexSettings(true, "parquet"); - DataformatAwareLockableWriterPool pool = stubPool(); + IndexSettings indexSettings = createIndexSettings("parquet"); IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool) + () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null) ); assertTrue(ex.getMessage().contains("parquet")); } @@ -82,7 +61,6 @@ public void testConstructorThrowsWhenSecondaryFormatNotRegistered() { plugins.put("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); Settings settings = Settings.builder() - .put("index.composite.enabled", true) .put("index.composite.primary_data_format", "lucene") .putList("index.composite.secondary_data_formats", "parquet") .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -92,24 +70,21 @@ public void testConstructorThrowsWhenSecondaryFormatNotRegistered() { IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); - DataformatAwareLockableWriterPool pool = stubPool(); IllegalArgumentException ex = expectThrows( IllegalArgumentException.class, - () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, pool) + () -> new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null) ); assertTrue(ex.getMessage().contains("parquet")); } public void testConstructorRejectsNullDataFormatPlugins() { - IndexSettings indexSettings = createIndexSettings(true, "lucene"); - DataformatAwareLockableWriterPool pool = stubPool(); - expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(null, indexSettings, null, null, pool)); + IndexSettings indexSettings = createIndexSettings("lucene"); + expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(null, indexSettings, null, null)); } public void testConstructorRejectsNullIndexSettings() { Map plugins = Map.of("lucene", CompositeTestHelper.stubPlugin("lucene", 1)); - DataformatAwareLockableWriterPool pool = stubPool(); - expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(plugins, null, null, null, pool)); + expectThrows(NullPointerException.class, () -> new CompositeIndexingExecutionEngine(plugins, null, null, null)); } public void testValidateFormatsRegisteredAcceptsValidConfig() { @@ -193,9 +168,8 @@ public void testDeleteFilesDoesNotThrow() throws Exception { engine.deleteFiles(Map.of()); } - private IndexSettings createIndexSettings(boolean compositeEnabled, String primaryFormat) { + private IndexSettings createIndexSettings(String primaryFormat) { Settings settings = Settings.builder() - .put("index.composite.enabled", compositeEnabled) .put("index.composite.primary_data_format", primaryFormat) .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) @@ -204,12 +178,4 @@ private IndexSettings createIndexSettings(boolean compositeEnabled, String prima IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); return new IndexSettings(indexMetadata, Settings.EMPTY); } - - /** - * Creates an uninitialized pool for tests that only exercise constructor validation - * and expect exceptions before the pool is ever used. - */ - private static DataformatAwareLockableWriterPool stubPool() { - return new DataformatAwareLockableWriterPool<>(ConcurrentLinkedQueue::new, 1); - } } diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java index 3e03b4fc0222c..d48ad5adb50f5 100644 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeTestHelper.java @@ -10,12 +10,10 @@ import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Nullable; import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.dataformat.DataFormat; import org.opensearch.index.engine.dataformat.DataFormatPlugin; -import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; import org.opensearch.index.engine.dataformat.DocumentInput; import org.opensearch.index.engine.dataformat.FieldTypeCapabilities; import org.opensearch.index.engine.dataformat.FileInfos; @@ -33,7 +31,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; /** * Shared test utilities for composite engine tests. @@ -53,7 +50,6 @@ static CompositeIndexingExecutionEngine createStubEngine(String primaryName, Str } Settings.Builder settingsBuilder = Settings.builder() - .put("index.composite.enabled", true) .put("index.composite.primary_data_format", primaryName) .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) @@ -67,12 +63,7 @@ static CompositeIndexingExecutionEngine createStubEngine(String primaryName, Str IndexMetadata indexMetadata = IndexMetadata.builder("test-index").settings(settings).build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); - DataformatAwareLockableWriterPool writerPool = new DataformatAwareLockableWriterPool<>( - ConcurrentLinkedQueue::new, - Runtime.getRuntime().availableProcessors() - ); - - return new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null, writerPool); + return new CompositeIndexingExecutionEngine(plugins, indexSettings, null, null); } static DataFormatPlugin stubPlugin(String formatName, long priority) { @@ -87,8 +78,7 @@ public DataFormat getDataFormat() { public IndexingExecutionEngine indexingEngine( MapperService mapperService, ShardPath shardPath, - IndexSettings indexSettings, - @Nullable DataformatAwareLockableWriterPool writerPool + IndexSettings indexSettings ) { return new StubIndexingExecutionEngine(format); } @@ -107,8 +97,7 @@ public DataFormat getDataFormat() { public IndexingExecutionEngine indexingEngine( MapperService mapperService, ShardPath shardPath, - IndexSettings indexSettings, - @Nullable DataformatAwareLockableWriterPool writerPool + IndexSettings indexSettings ) { return new StubIndexingExecutionEngine(format); } diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java index c34f28fca0ca0..a5c18f7cd3f4b 100644 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeWriterTests.java @@ -36,6 +36,7 @@ public void testWriterGenerationIsPreserved() throws IOException { public void testAbortedDefaultsToFalse() throws IOException { CompositeWriter writer = new CompositeWriter(engine, 0); assertFalse(writer.isAborted()); + assertEquals(CompositeWriter.WriterState.ACTIVE, writer.getState()); writer.close(); } @@ -43,12 +44,14 @@ public void testAbortSetsAbortedFlag() throws IOException { CompositeWriter writer = new CompositeWriter(engine, 0); writer.abort(); assertTrue(writer.isAborted()); + assertEquals(CompositeWriter.WriterState.ABORTED, writer.getState()); writer.close(); } public void testFlushPendingDefaultsToFalse() throws IOException { CompositeWriter writer = new CompositeWriter(engine, 0); assertFalse(writer.isFlushPending()); + assertEquals(CompositeWriter.WriterState.ACTIVE, writer.getState()); writer.close(); } @@ -56,6 +59,25 @@ public void testSetFlushPendingSetsFlag() throws IOException { CompositeWriter writer = new CompositeWriter(engine, 0); writer.setFlushPending(); assertTrue(writer.isFlushPending()); + assertEquals(CompositeWriter.WriterState.FLUSH_PENDING, writer.getState()); + writer.close(); + } + + public void testAbortDoesNotTransitionFromFlushPending() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.setFlushPending(); + expectThrows(IllegalStateException.class, writer::abort); + assertTrue(writer.isFlushPending()); + assertEquals(CompositeWriter.WriterState.FLUSH_PENDING, writer.getState()); + writer.close(); + } + + public void testFlushPendingDoesNotTransitionFromAborted() throws IOException { + CompositeWriter writer = new CompositeWriter(engine, 0); + writer.abort(); + expectThrows(IllegalStateException.class, writer::setFlushPending); + assertTrue(writer.isAborted()); + assertEquals(CompositeWriter.WriterState.ABORTED, writer.getState()); writer.close(); } diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java deleted file mode 100644 index 2dbe050d517be..0000000000000 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/DataformatAwareLockableWriterPoolTests.java +++ /dev/null @@ -1,137 +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. - */ - -package org.opensearch.composite; - -import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.index.engine.dataformat.DataformatAwareLockableWriterPool; -import org.opensearch.test.OpenSearchTestCase; - -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Tests for {@link DataformatAwareLockableWriterPool} with composite-specific callbacks. - */ -public class DataformatAwareLockableWriterPoolTests extends OpenSearchTestCase { - - private CompositeIndexingExecutionEngine engine; - - @Override - public void setUp() throws Exception { - super.setUp(); - engine = CompositeTestHelper.createStubEngine("lucene"); - } - - private DataformatAwareLockableWriterPool createPool(int concurrency) { - AtomicLong gen = new AtomicLong(0); - DataformatAwareLockableWriterPool pool = new DataformatAwareLockableWriterPool<>( - ConcurrentLinkedQueue::new, - concurrency - ); - pool.initialize(() -> new CompositeWriter(engine, gen.getAndIncrement())); - return pool; - } - - public void testGetAndLockReturnsNewWriterWhenPoolEmpty() { - DataformatAwareLockableWriterPool pool = createPool(2); - - CompositeWriter writer = pool.getAndLock(); - assertNotNull(writer); - assertEquals(0L, writer.getWriterGeneration()); - pool.releaseAndUnlock(writer); - } - - public void testReleaseAndUnlockMakesWriterReusable() { - DataformatAwareLockableWriterPool pool = createPool(2); - - CompositeWriter writer = pool.getAndLock(); - pool.releaseAndUnlock(writer); - - CompositeWriter reused = pool.getAndLock(); - assertSame(writer, reused); - pool.releaseAndUnlock(reused); - } - - public void testGetAndLockThrowsWhenClosed() throws IOException { - DataformatAwareLockableWriterPool pool = createPool(2); - pool.close(); - expectThrows(AlreadyClosedException.class, pool::getAndLock); - } - - public void testCheckoutAllReturnsRegisteredWriters() { - DataformatAwareLockableWriterPool pool = createPool(4); - - CompositeWriter w1 = pool.getAndLock(); - CompositeWriter w2 = pool.getAndLock(); - pool.releaseAndUnlock(w1); - pool.releaseAndUnlock(w2); - - List checkedOut = pool.checkoutAll(); - assertEquals(2, checkedOut.size()); - for (CompositeWriter w : checkedOut) { - assertFalse(w.isFlushPending()); - } - } - - public void testCheckoutAllReturnsEmptyWhenNoWriters() { - DataformatAwareLockableWriterPool pool = createPool(2); - - List checkedOut = pool.checkoutAll(); - assertTrue(checkedOut.isEmpty()); - } - - public void testCheckoutAllReturnsUnmodifiableList() { - DataformatAwareLockableWriterPool pool = createPool(2); - - CompositeWriter w = pool.getAndLock(); - pool.releaseAndUnlock(w); - - List checkedOut = pool.checkoutAll(); - expectThrows(UnsupportedOperationException.class, () -> checkedOut.add(new CompositeWriter(engine, 99))); - } - - public void testIsRegisteredReturnsTrueForPooledWriter() { - DataformatAwareLockableWriterPool pool = createPool(2); - - CompositeWriter writer = pool.getAndLock(); - assertTrue(pool.isRegistered(writer)); - pool.releaseAndUnlock(writer); - } - - public void testIsRegisteredReturnsFalseForUnknownWriter() { - DataformatAwareLockableWriterPool pool = createPool(2); - - CompositeWriter unknown = new CompositeWriter(engine, 99); - assertFalse(pool.isRegistered(unknown)); - } - - public void testCheckoutAllThrowsWhenClosed() throws IOException { - DataformatAwareLockableWriterPool pool = createPool(2); - pool.close(); - expectThrows(AlreadyClosedException.class, pool::checkoutAll); - } - - public void testIteratorReturnsSnapshotOfWriters() { - DataformatAwareLockableWriterPool pool = createPool(4); - - CompositeWriter w1 = pool.getAndLock(); - CompositeWriter w2 = pool.getAndLock(); - pool.releaseAndUnlock(w1); - pool.releaseAndUnlock(w2); - - int count = 0; - for (CompositeWriter w : pool) { - assertNotNull(w); - count++; - } - assertEquals(2, count); - } -} diff --git a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java index 3ea6bdbd36fba..9c28cf4cb825b 100644 --- a/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java +++ b/server/src/main/java/org/opensearch/index/engine/dataformat/DataFormatPlugin.java @@ -8,12 +8,18 @@ package org.opensearch.index.engine.dataformat; -import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardPath; +/** + * Plugin interface for providing custom data format implementations. + * Plugins implement this to register their data format (e.g., Parquet, Lucene) + * with the DataFormatRegistry during node bootstrap. + * + * @opensearch.experimental + */ /** * Plugin interface for providing custom data format implementations. * Plugins implement this to register their data format (e.g., Parquet, Lucene) @@ -37,14 +43,7 @@ public interface DataFormatPlugin { * @param mapperService the mapper service for field mapping resolution * @param shardPath the shard path for file storage * @param indexSettings the index settings - * @param writerPool the writer pool for managing writer instances, or {@code null} if the engine - * does not require external writer pool management (e.g., individual format engines) * @return the indexing execution engine instance */ - IndexingExecutionEngine indexingEngine( - MapperService mapperService, - ShardPath shardPath, - IndexSettings indexSettings, - @Nullable DataformatAwareLockableWriterPool writerPool - ); + IndexingExecutionEngine indexingEngine(MapperService mapperService, ShardPath shardPath, IndexSettings indexSettings); } diff --git a/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java b/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java deleted file mode 100644 index f29cf2345ab37..0000000000000 --- a/server/src/main/java/org/opensearch/index/engine/dataformat/DataformatAwareLockableWriterPool.java +++ /dev/null @@ -1,166 +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. - */ - -package org.opensearch.index.engine.dataformat; - -import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.common.SetOnce; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.queue.Lockable; -import org.opensearch.common.queue.LockableConcurrentQueue; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.IdentityHashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.Queue; -import java.util.Set; -import java.util.function.Supplier; - -/** - * A data-format-aware pool of {@link Writer} instances backed by a {@link LockableConcurrentQueue}. - * Writers are locked on checkout and unlocked on release, ensuring thread-safe reuse. - *

- * The pool is created with queue configuration only. The writer supplier is set later - * via {@link #initialize(Supplier)}, allowing the engine that owns the pool to provide - * the writer factory after construction (avoiding circular dependencies). - * - * @param the writer type, must implement both {@link Writer} and {@link Lockable} - * - * @opensearch.experimental - */ -@ExperimentalApi -public final class DataformatAwareLockableWriterPool & Lockable> implements Iterable, Closeable { - - private final Set writers; - private final LockableConcurrentQueue availableWriters; - private final SetOnce closed = new SetOnce<>(); - private volatile Supplier writerSupplier; - - /** - * Creates a new writer pool. The pool must be initialized - * with a writer supplier via {@link #initialize(Supplier)} before use. - * - * @param queueSupplier supplier for the underlying queue instances - * @param concurrency the concurrency level (number of stripes) - */ - public DataformatAwareLockableWriterPool(Supplier> queueSupplier, int concurrency) { - this.writers = Collections.newSetFromMap(new IdentityHashMap<>()); - this.availableWriters = new LockableConcurrentQueue<>(queueSupplier, concurrency); - } - - /** - * Initializes the pool with a writer supplier. Must be called exactly once before - * any checkout operations. - * - * @param writerSupplier factory for creating new writer instances - * @throws IllegalStateException if already initialized - */ - public void initialize(Supplier writerSupplier) { - if (this.writerSupplier != null) { - throw new IllegalStateException("DataformatAwareLockableWriterPool is already initialized"); - } - this.writerSupplier = Objects.requireNonNull(writerSupplier, "writerSupplier must not be null"); - } - - /** - * Locks and polls a writer from the pool, or creates a new one if none are available. - * - * @return a locked writer instance - */ - public W getAndLock() { - ensureOpen(); - ensureInitialized(); - W writer = availableWriters.lockAndPoll(); - return Objects.requireNonNullElseGet(writer, this::fetchWriter); - } - - private synchronized W fetchWriter() { - ensureOpen(); - W writer = writerSupplier.get(); - writer.lock(); - writers.add(writer); - return writer; - } - - /** - * Releases the given writer back to this pool for reuse. - * - * @param writer the writer to release - */ - public void releaseAndUnlock(W writer) { - assert isRegistered(writer) : "WriterPool doesn't know about this writer"; - availableWriters.addAndUnlock(writer); - } - - /** - * Lock and checkout all writers from the pool for flush. - * - * @return unmodifiable list of all writers locked by current thread - */ - public List checkoutAll() { - ensureOpen(); - List lockedWriters = new ArrayList<>(); - List checkedOutWriters = new ArrayList<>(); - for (W writer : this) { - writer.lock(); - lockedWriters.add(writer); - } - synchronized (this) { - for (W writer : lockedWriters) { - try { - if (isRegistered(writer) && writers.remove(writer)) { - availableWriters.remove(writer); - checkedOutWriters.add(writer); - } - } finally { - writer.unlock(); - } - } - } - return Collections.unmodifiableList(checkedOutWriters); - } - - /** - * Check if a writer is part of this pool. - * - * @param writer writer to validate - * @return true if the writer is part of this pool - */ - public synchronized boolean isRegistered(W writer) { - return writers.contains(writer); - } - - private void ensureOpen() { - if (closed.get() != null) { - throw new AlreadyClosedException("DataformatAwareLockableWriterPool is already closed"); - } - } - - private void ensureInitialized() { - if (writerSupplier == null) { - throw new IllegalStateException("DataformatAwareLockableWriterPool has not been initialized with a writer supplier"); - } - } - - @Override - public synchronized Iterator iterator() { - return List.copyOf(writers).iterator(); - } - - @Override - public void close() throws IOException { - if (closed.trySet(true) == false) { - throw new AlreadyClosedException("DataformatAwareLockableWriterPool is already closed"); - } - } -} diff --git a/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java b/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java index 99f0ea59f4067..a78a5616ac23f 100644 --- a/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java +++ b/server/src/test/java/org/opensearch/index/engine/dataformat/DataFormatPluginTests.java @@ -70,8 +70,7 @@ public void testFullDataFormatLifecycle() throws IOException { .indexingEngine( mock(MapperService.class), new ShardPath(false, Path.of("/tmp/uuid/0"), Path.of("/tmp/uuid/0"), new ShardId("index", "uuid", 0)), - new IndexSettings(IndexMetadata.builder("index").settings(settings).build(), settings), - null + new IndexSettings(IndexMetadata.builder("index").settings(settings).build(), settings) ); assertEquals(format, engine.getDataFormat()); @@ -406,12 +405,7 @@ public DataFormat getDataFormat() { } @Override - public IndexingExecutionEngine indexingEngine( - MapperService mapperService, - ShardPath shardPath, - IndexSettings indexSettings, - DataformatAwareLockableWriterPool writerPool - ) { + public IndexingExecutionEngine indexingEngine(MapperService mapperService, ShardPath shardPath, IndexSettings indexSettings) { return new MockIndexingExecutionEngine(dataFormat); } } From 7fb1568b4cbf024cd25f52011189f3aa5429bd38 Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Wed, 25 Mar 2026 16:33:06 +0530 Subject: [PATCH 3/9] Remove auxilary index from the ciee Signed-off-by: Arpit Bandejiya --- CHANGELOG.md | 1 + .../common/queue/ConcurrentQueue.java | 25 ------------------- 2 files changed, 1 insertion(+), 25 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d51ae43ec81f7..25bbb6134a5dc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,6 +24,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add new setting property 'Sensitive' for tiering dynamic settings ([#20901](https://github.com/opensearch-project/OpenSearch/pull/20901)) - Add ref_path support for package-based hunspell dictionary loading ([#20840](https://github.com/opensearch-project/OpenSearch/pull/20840)) - Add support for enabling pluggable data formats, starting with phase-1 of decoupling shard from engine, and introducing basic abstractions ([#20675](https://github.com/opensearch-project/OpenSearch/pull/20675)) +- Add concurrent queue in libs and composite engine sandbox plugin ([#20909](https://github.com/opensearch-project/OpenSearch/pull/20909)) - Add warmup phase to wait for lag to catch up in pull-based ingestion before serving ([#20526](https://github.com/opensearch-project/OpenSearch/pull/20526)) - Add a new static method to IndicesOptions API to expose `STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED` index option ([#20980](https://github.com/opensearch-project/OpenSearch/pull/20980)) diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java index 06fa238b4ad93..f7856c9464f54 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java @@ -10,7 +10,6 @@ import java.util.Iterator; import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; @@ -33,8 +32,6 @@ public final class ConcurrentQueue { private final Lock[] locks; private final Queue[] queues; private final Supplier> queueSupplier; - /** Maps each entry to its queue index so that {@link #remove} can go directly to the right queue. */ - private final ConcurrentHashMap queueIndex; ConcurrentQueue(Supplier> queueSupplier, int concurrency) { if (concurrency < MIN_CONCURRENCY || concurrency > MAX_CONCURRENCY) { @@ -44,7 +41,6 @@ public final class ConcurrentQueue { } this.concurrency = concurrency; this.queueSupplier = queueSupplier; - this.queueIndex = new ConcurrentHashMap<>(); locks = new Lock[concurrency]; @SuppressWarnings({ "rawtypes", "unchecked" }) Queue[] queues = new Queue[concurrency]; @@ -67,7 +63,6 @@ void add(T entry) { if (lock.tryLock()) { try { queue.add(entry); - queueIndex.put(entry, index); return; } finally { lock.unlock(); @@ -80,7 +75,6 @@ void add(T entry) { lock.lock(); try { queue.add(entry); - queueIndex.put(entry, index); } finally { lock.unlock(); } @@ -99,7 +93,6 @@ T poll(Predicate predicate) { T entry = it.next(); if (predicate.test(entry)) { it.remove(); - queueIndex.remove(entry); return entry; } } @@ -119,7 +112,6 @@ T poll(Predicate predicate) { T entry = it.next(); if (predicate.test(entry)) { it.remove(); - queueIndex.remove(entry); return entry; } } @@ -131,29 +123,12 @@ T poll(Predicate predicate) { } boolean remove(T entry) { - Integer queueIdx = queueIndex.get(entry); - if (queueIdx != null) { - final Lock lock = locks[queueIdx]; - final Queue queue = queues[queueIdx]; - lock.lock(); - try { - if (queue.remove(entry)) { - queueIndex.remove(entry); - return true; - } - } finally { - lock.unlock(); - } - } - // Fallback: entry may have been re-added to a different queue between the index lookup - // and the lock acquisition, so scan all stripes. for (int i = 0; i < concurrency; ++i) { final Lock lock = locks[i]; final Queue queue = queues[i]; lock.lock(); try { if (queue.remove(entry)) { - queueIndex.remove(entry); return true; } } finally { From 807a89260f577e53bbbbe8007a0ea0f11980f569 Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Thu, 26 Mar 2026 14:44:49 +0530 Subject: [PATCH 4/9] Fix tests post merge Signed-off-by: Arpit Bandejiya --- .../org/opensearch/analytics/exec/DefaultPlanExecutorTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sandbox/plugins/analytics-engine/src/test/java/org/opensearch/analytics/exec/DefaultPlanExecutorTests.java b/sandbox/plugins/analytics-engine/src/test/java/org/opensearch/analytics/exec/DefaultPlanExecutorTests.java index 0b34e3ea6495b..86f1086f650ba 100644 --- a/sandbox/plugins/analytics-engine/src/test/java/org/opensearch/analytics/exec/DefaultPlanExecutorTests.java +++ b/sandbox/plugins/analytics-engine/src/test/java/org/opensearch/analytics/exec/DefaultPlanExecutorTests.java @@ -177,7 +177,7 @@ private static class StubTableScan extends TableScan { } } - static class MockDataFormat implements DataFormat { + static class MockDataFormat extends DataFormat { @Override public String name() { return "mock-columnar"; From 3a9ddb36cf64b6d97655b53228b385b0cfc17d78 Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Fri, 27 Mar 2026 00:16:27 +0530 Subject: [PATCH 5/9] address commits Signed-off-by: Arpit Bandejiya --- benchmarks/build.gradle | 1 - libs/concurrent-queue/build.gradle | 38 ++++++++++++++ .../LockableConcurrentQueueBenchmark.java | 31 ++---------- .../composite/CompositeDocumentInput.java | 8 +-- .../composite/CompositeEnginePlugin.java | 2 +- .../CompositeIndexingExecutionEngine.java | 34 ++++++++++--- .../opensearch/composite/CompositeWriter.java | 23 +++++---- .../CompositeDocumentInputTests.java | 49 ++++--------------- 8 files changed, 93 insertions(+), 93 deletions(-) rename {benchmarks/src/main/java/org/opensearch/benchmark => libs/concurrent-queue/src/jmh/java/org/opensearch/common}/queue/LockableConcurrentQueueBenchmark.java (73%) diff --git a/benchmarks/build.gradle b/benchmarks/build.gradle index 21c8ed7a7741e..732e77934b427 100644 --- a/benchmarks/build.gradle +++ b/benchmarks/build.gradle @@ -51,7 +51,6 @@ dependencies { // us to invoke the JMH uberjar as usual. exclude group: 'net.sf.jopt-simple', module: 'jopt-simple' } - api project(':libs:opensearch-concurrent-queue') api "org.openjdk.jmh:jmh-core:$versions.jmh" annotationProcessor "org.openjdk.jmh:jmh-generator-annprocess:$versions.jmh" // Dependencies of JMH diff --git a/libs/concurrent-queue/build.gradle b/libs/concurrent-queue/build.gradle index 6a4cd5c909bb8..e4f1550378c77 100644 --- a/libs/concurrent-queue/build.gradle +++ b/libs/concurrent-queue/build.gradle @@ -11,6 +11,15 @@ * No external dependencies — pure Java concurrency primitives. */ +// ---- JMH benchmark source set ---- +sourceSets { + jmh { + java.srcDirs = ['src/jmh/java'] + compileClasspath += sourceSets.main.output + runtimeClasspath += sourceSets.main.output + } +} + dependencies { /******* * !!!! NO RUNTIME DEPENDENCIES !!!! @@ -23,6 +32,10 @@ dependencies { testImplementation(project(":test:framework")) { exclude group: 'org.opensearch', module: 'opensearch-concurrent-queue' } + + // JMH benchmark dependencies + jmhImplementation "org.openjdk.jmh:jmh-core:${versions.jmh}" + jmhAnnotationProcessor "org.openjdk.jmh:jmh-generator-annprocess:${versions.jmh}" } testingConventions.enabled = true @@ -31,3 +44,28 @@ tasks.named('forbiddenApisMain').configure { // :libs:opensearch-concurrent-queue does not depend on server replaceSignatureFiles 'jdk-signatures' } + +// JMH generated code uses forbidden APIs — skip checking +tasks.matching { it.name == 'forbiddenApisJmh' }.configureEach { enabled = false } + +// JMH annotation processor +tasks.named('compileJmhJava').configure { + options.compilerArgs.addAll(["-processor", "org.openjdk.jmh.generators.BenchmarkProcessor"]) +} + +// Convenience task to run JMH benchmarks +tasks.register('jmh', JavaExec) { + dependsOn jmhClasses + mainClass = 'org.openjdk.jmh.Main' + classpath = sourceSets.jmh.runtimeClasspath + // Pass -Pjmh.includes= to filter benchmarks + if (project.hasProperty('jmh.includes')) { + args project.property('jmh.includes') + } +} + +spotless { + java { + targetExclude 'src/jmh/generated/**/*.java' + } +} diff --git a/benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockableConcurrentQueueBenchmark.java similarity index 73% rename from benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java rename to libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockableConcurrentQueueBenchmark.java index cae90dae0b4d2..510f95abc551b 100644 --- a/benchmarks/src/main/java/org/opensearch/benchmark/queue/LockableConcurrentQueueBenchmark.java +++ b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockableConcurrentQueueBenchmark.java @@ -6,10 +6,8 @@ * compatible open source license. */ -package org.opensearch.benchmark.queue; +package org.opensearch.common.queue; -import org.opensearch.common.queue.Lockable; -import org.opensearch.common.queue.LockableConcurrentQueue; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -32,14 +30,6 @@ /** * JMH benchmark for {@link LockableConcurrentQueue} measuring throughput of * lock-and-poll / add-and-unlock cycles under varying concurrency levels. - *

- * Includes two benchmark groups: - *

    - *
  • {@code pollAndReturn} — minimal overhead: poll an entry and immediately return it.
  • - *
  • {@code writerWorkload} — simulates a writer pool: poll an entry, perform simulated - * document writes (CPU work), then return the entry. Models the composite writer - * checkout-write-return cycle.
  • - *
*/ @Fork(3) @Warmup(iterations = 5, time = 1) @@ -68,8 +58,6 @@ public void setup() { } } - // ---- pollAndReturn: minimal overhead benchmarks ---- - @Benchmark @Threads(1) public LockableEntry pollAndReturn_1thread() { @@ -96,8 +84,6 @@ private LockableEntry pollAndReturn() { return entry; } - // ---- writerWorkload: simulated writer pool benchmarks ---- - @Benchmark @Threads(4) public void writerWorkload_4threads(Blackhole bh) { @@ -116,27 +102,16 @@ public void writerWorkload_16threads(Blackhole bh) { writerWorkload(bh); } - /** - * Simulates a writer pool cycle: checkout an entry, perform CPU work - * representing document indexing across multiple formats, then return it. - */ private void writerWorkload(Blackhole bh) { LockableEntry entry = queue.lockAndPoll(); if (entry != null) { - // Simulate document write work (field additions across formats) - bh.consume(simulateDocumentWrite(entry)); + bh.consume(simulateWork(entry)); queue.addAndUnlock(entry); } } - /** - * Simulates the CPU cost of writing a document to multiple data formats. - * Performs arithmetic work to prevent JIT elimination while keeping - * the hold time realistic relative to a real addDoc call. - */ - private static long simulateDocumentWrite(LockableEntry entry) { + private static long simulateWork(LockableEntry entry) { long result = entry.hashCode(); - // ~10 field additions across 2 formats for (int i = 0; i < 20; i++) { result ^= (result << 13); result ^= (result >> 7); diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java index d16e909dc64f1..75d7b37368f0c 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java @@ -32,7 +32,6 @@ public class CompositeDocumentInput implements DocumentInput primaryDocumentInput; private final DataFormat primaryFormat; private final Map> secondaryDocumentInputs; - private final Runnable onClose; /** * Constructs a CompositeDocumentInput with a primary format input and secondary format inputs. @@ -40,20 +39,17 @@ public class CompositeDocumentInput implements DocumentInput primaryDocumentInput, - Map> secondaryDocumentInputs, - Runnable onClose + Map> secondaryDocumentInputs ) { this.primaryFormat = Objects.requireNonNull(primaryFormat, "primaryFormat must not be null"); this.primaryDocumentInput = Objects.requireNonNull(primaryDocumentInput, "primaryDocumentInput must not be null"); this.secondaryDocumentInputs = Map.copyOf( Objects.requireNonNull(secondaryDocumentInputs, "secondaryDocumentInputs must not be null") ); - this.onClose = Objects.requireNonNull(onClose, "onClose must not be null"); } @Override @@ -93,7 +89,7 @@ public List> getFinalInput() { @Override public void close() { - onClose.run(); + // No-op: document input lifecycle is independent of writer pool } /** diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java index c87bf339beb0d..182171c499ff0 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeEnginePlugin.java @@ -122,7 +122,7 @@ public void loadExtensions(ExtensionLoader loader) { registry.put(name, plugin); logger.info("Registered DataFormatPlugin [{}] for format [{}]", plugin.getClass().getName(), name); } - this.dataFormatPlugins = Collections.unmodifiableMap(registry); + this.dataFormatPlugins = Map.copyOf(registry); } @Override diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java index 42c6f4e5a702e..b1080cbe9a63c 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java @@ -134,6 +134,9 @@ static void validateFormatsRegistered( String primaryFormatName, List secondaryFormatNames ) { + if (primaryFormatName == null || primaryFormatName.isBlank()) { + throw new IllegalArgumentException("Primary data format name must not be null or blank"); + } if (dataFormatPlugins.containsKey(primaryFormatName) == false) { throw new IllegalArgumentException( "Primary data format [" @@ -143,6 +146,9 @@ static void validateFormatsRegistered( ); } for (String secondaryName : secondaryFormatNames) { + if (secondaryName == null || secondaryName.isBlank()) { + throw new IllegalArgumentException("Secondary data format name must not be null or blank"); + } if (secondaryName.equals(primaryFormatName)) { throw new IllegalStateException( "Secondary data format [" + secondaryName + "] is the same as primary :[" + primaryFormatName + "]" @@ -242,24 +248,38 @@ public long getNativeBytesUsed() { @Override public void deleteFiles(Map> filesToDelete) throws IOException { - primaryEngine.deleteFiles(filesToDelete); + IOException firstException = null; + try { + primaryEngine.deleteFiles(filesToDelete); + } catch (IOException e) { + logger.error("Failed to delete files in primary engine [{}]: {}", primaryEngine.getDataFormat().name(), e.getMessage()); + firstException = e; + } for (IndexingExecutionEngine engine : secondaryEngines) { - engine.deleteFiles(filesToDelete); + try { + engine.deleteFiles(filesToDelete); + } catch (IOException e) { + logger.error("Failed to delete files in secondary engine [{}]: {}", engine.getDataFormat().name(), e.getMessage()); + if (firstException == null) { + firstException = e; + } else { + firstException.addSuppressed(e); + } + } + } + if (firstException != null) { + throw firstException; } } @Override public CompositeDocumentInput newDocumentInput() { - CompositeWriter writer = writerPool.getAndLock(); DocumentInput primaryInput = primaryEngine.newDocumentInput(); Map> secondaryInputMap = new LinkedHashMap<>(); for (IndexingExecutionEngine engine : secondaryEngines) { secondaryInputMap.put(engine.getDataFormat(), engine.newDocumentInput()); } - return new CompositeDocumentInput(primaryEngine.getDataFormat(), primaryInput, secondaryInputMap, () -> { - assert writer.getState() == CompositeWriter.WriterState.ACTIVE : "CompositeWriter is not ACTIVE, state=" + writer.getState(); - writerPool.releaseAndUnlock(writer); - }); + return new CompositeDocumentInput(primaryEngine.getDataFormat(), primaryInput, secondaryInputMap); } /** diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java index d36bbaae8a3cc..309494adf21e9 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java @@ -40,7 +40,7 @@ * @opensearch.experimental */ @ExperimentalApi -public class CompositeWriter implements Writer, Lockable { +class CompositeWriter implements Writer, Lockable { private static final Logger logger = LogManager.getLogger(CompositeWriter.class); @@ -61,7 +61,7 @@ public class CompositeWriter implements Writer, Lockable * */ @ExperimentalApi - public enum WriterState { + enum WriterState { /** Writer is actively accepting documents. */ ACTIVE, /** Writer has been marked for flushing and should not accept new documents. */ @@ -81,7 +81,7 @@ public enum WriterState { * @param writerGeneration the writer generation number */ @SuppressWarnings("unchecked") - public CompositeWriter(CompositeIndexingExecutionEngine engine, long writerGeneration) { + CompositeWriter(CompositeIndexingExecutionEngine engine, long writerGeneration) { this.lock = new ReentrantLock(); this.state = new AtomicReference<>(WriterState.ACTIVE); this.writerGeneration = writerGeneration; @@ -102,6 +102,9 @@ public CompositeWriter(CompositeIndexingExecutionEngine engine, long writerGener @Override public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { + if (state.get() != WriterState.ACTIVE) { + throw new IllegalStateException("Cannot add document to writer in state " + state.get()); + } // Write to primary first WriteResult primaryResult = primaryWriter.getValue().addDoc(doc.getPrimaryInput()); switch (primaryResult) { @@ -172,7 +175,7 @@ public void close() throws IOException { * * @return the writer generation */ - public long getWriterGeneration() { + long getWriterGeneration() { return writerGeneration; } @@ -181,7 +184,7 @@ public long getWriterGeneration() { * * @throws IllegalStateException if the writer is not in {@code ACTIVE} state */ - public void abort() { + void abort() { if (this.state.compareAndSet(WriterState.ACTIVE, WriterState.ABORTED) == false) { throw new IllegalStateException("Cannot abort writer in state " + state.get()); } @@ -192,7 +195,7 @@ public void abort() { * * @return {@code true} if aborted */ - public boolean isAborted() { + boolean isAborted() { return getState() == WriterState.ABORTED; } @@ -201,7 +204,7 @@ public boolean isAborted() { * * @throws IllegalStateException if the writer is not in {@code ACTIVE} state */ - public void setFlushPending() { + void setFlushPending() { if (this.state.compareAndSet(WriterState.ACTIVE, WriterState.FLUSH_PENDING) == false) { throw new IllegalStateException("Cannot set flush pending on writer in state " + state.get()); } @@ -212,16 +215,16 @@ public void setFlushPending() { * * @return {@code true} if a flush is pending */ - public boolean isFlushPending() { + boolean isFlushPending() { return getState() == WriterState.FLUSH_PENDING; } /** - * Returns the current state of this writer. + * Returns the current state of this writer for testing purpose. * * @return the writer state */ - public WriterState getState() { + WriterState getState() { return state.get(); } diff --git a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java index 958532689fdd7..979910a7b739f 100644 --- a/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java +++ b/sandbox/plugins/composite-engine/src/test/java/org/opensearch/composite/CompositeDocumentInputTests.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; /** * Tests for {@link CompositeDocumentInput}. @@ -38,12 +37,7 @@ public void testAddFieldBroadcastsToAllFormats() { DataFormat primaryFormat = mockFormat("lucene", 1, Set.of()); DataFormat secondaryFormat = mockFormat("parquet", 2, Set.of()); - CompositeDocumentInput composite = new CompositeDocumentInput( - primaryFormat, - primaryInput, - Map.of(secondaryFormat, secondaryInput), - () -> {} - ); + CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, primaryInput, Map.of(secondaryFormat, secondaryInput)); MappedFieldType keywordField = mockFieldType("keyword"); composite.addField(keywordField, "value1"); @@ -65,7 +59,7 @@ public void testSetRowIdBroadcastsToAllInputs() { secondaries.put(secondaryFormat1, secondary1); secondaries.put(secondaryFormat2, secondary2); - CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, primaryInput, secondaries, () -> {}); + CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, primaryInput, secondaries); composite.setRowId("_row_id", 42L); @@ -81,34 +75,20 @@ public void testGetFinalInputReturnsNull() { CompositeDocumentInput composite = new CompositeDocumentInput( mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), - Map.of(), - () -> {} + Map.of() ); assertNull(composite.getFinalInput()); } - public void testCloseInvokesOnCloseCallback() { - AtomicBoolean closed = new AtomicBoolean(false); - CompositeDocumentInput composite = new CompositeDocumentInput( - mockFormat("lucene", 1, Set.of()), - new RecordingDocumentInput(), - Map.of(), - () -> closed.set(true) - ); - - composite.close(); - assertTrue(closed.get()); - } - public void testGetPrimaryInputReturnsPrimaryDocumentInput() { RecordingDocumentInput primaryInput = new RecordingDocumentInput(); - CompositeDocumentInput composite = new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), primaryInput, Map.of(), () -> {}); + CompositeDocumentInput composite = new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), primaryInput, Map.of()); assertSame(primaryInput, composite.getPrimaryInput()); } public void testGetPrimaryFormatReturnsPrimaryDataFormat() { DataFormat primaryFormat = mockFormat("lucene", 1, Set.of()); - CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, new RecordingDocumentInput(), Map.of(), () -> {}); + CompositeDocumentInput composite = new CompositeDocumentInput(primaryFormat, new RecordingDocumentInput(), Map.of()); assertSame(primaryFormat, composite.getPrimaryFormat()); } @@ -119,8 +99,7 @@ public void testGetSecondaryInputsReturnsUnmodifiableMap() { CompositeDocumentInput composite = new CompositeDocumentInput( mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), - Map.of(secondaryFormat, secondaryInput), - () -> {} + Map.of(secondaryFormat, secondaryInput) ); Map> secondaries = composite.getSecondaryInputs(); @@ -132,27 +111,17 @@ public void testGetSecondaryInputsReturnsUnmodifiableMap() { } public void testConstructorRejectsNullPrimaryFormat() { - expectThrows(NullPointerException.class, () -> new CompositeDocumentInput(null, new RecordingDocumentInput(), Map.of(), () -> {})); + expectThrows(NullPointerException.class, () -> new CompositeDocumentInput(null, new RecordingDocumentInput(), Map.of())); } public void testConstructorRejectsNullPrimaryInput() { - expectThrows( - NullPointerException.class, - () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), null, Map.of(), () -> {}) - ); + expectThrows(NullPointerException.class, () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), null, Map.of())); } public void testConstructorRejectsNullSecondaryInputs() { expectThrows( NullPointerException.class, - () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), null, () -> {}) - ); - } - - public void testConstructorRejectsNullOnClose() { - expectThrows( - NullPointerException.class, - () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), Map.of(), null) + () -> new CompositeDocumentInput(mockFormat("lucene", 1, Set.of()), new RecordingDocumentInput(), null) ); } From c5a4fc5e537f2b5de27e0d942ae654768af85fd4 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sat, 28 Mar 2026 22:03:46 +0530 Subject: [PATCH 6/9] Simplify the locks during refresh, use IdentifyHashMap Signed-off-by: Bukhtawar Khan --- .../opensearch/common/queue/LockablePool.java | 59 ++++++++++++------- .../composite/CompositeDocumentInput.java | 3 +- .../CompositeIndexingExecutionEngine.java | 4 +- .../opensearch/composite/CompositeWriter.java | 31 +++++----- 4 files changed, 57 insertions(+), 40 deletions(-) diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java index 505c92abd5833..d11b7dec9ff4e 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java @@ -31,9 +31,11 @@ */ public final class LockablePool implements Iterable, Closeable { - private final Set items; - private final LockableConcurrentQueue availableItems; + private volatile Set items; + private volatile LockableConcurrentQueue availableItems; private final Supplier itemSupplier; + private final Supplier> queueSupplier; + private final int concurrency; private volatile boolean closed; /** @@ -46,6 +48,8 @@ public final class LockablePool implements Iterable, Clos public LockablePool(Supplier itemSupplier, Supplier> queueSupplier, int concurrency) { this.items = Collections.newSetFromMap(new IdentityHashMap<>()); this.itemSupplier = Objects.requireNonNull(itemSupplier, "itemSupplier must not be null"); + this.queueSupplier = Objects.requireNonNull(queueSupplier, "queueSupplier must not be null"); + this.concurrency = concurrency; this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); } @@ -70,39 +74,52 @@ private synchronized T fetchItem() { } /** - * Releases the given item back to this pool for reuse. + * Releases the given item back to this pool for reuse. If the item belongs to a previous + * generation (swapped out during {@link #checkoutAll()}), it is silently unlocked and + * discarded since the checkout caller owns it. * * @param item the item to release */ public void releaseAndUnlock(T item) { - assert isRegistered(item) : "Pool doesn't know about this item"; + if (isRegistered(item) == false) { + // Item belongs to a previous generation swapped out during checkoutAll(). + // Just unlock it — the checkout caller owns it now. + item.unlock(); + return; + } availableItems.addAndUnlock(item); } /** - * Lock and checkout all items from the pool. + * Atomically swaps the pool's item set and queue with fresh instances, then waits for + * any in-flight operations on the old items to complete. This minimizes the time the pool + * lock is held — callers of {@link #getAndLock()} see the new empty pool immediately and + * can create fresh items without waiting for the checkout to finish. * - * @return unmodifiable list of all items locked by current thread + * @return unmodifiable list of all checked-out items * @throws IllegalStateException if the pool is closed */ public List checkoutAll() { ensureOpen(); - List lockedItems = new ArrayList<>(); - List checkedOutItems = new ArrayList<>(); - for (T item : this) { - item.lock(); - lockedItems.add(item); - } + + // Step 1: Atomic swap — hold pool lock only for the reference swap. + Set oldItems; synchronized (this) { - for (T item : lockedItems) { - try { - if (isRegistered(item) && items.remove(item)) { - availableItems.remove(item); - checkedOutItems.add(item); - } - } finally { - item.unlock(); - } + oldItems = this.items; + this.items = Collections.newSetFromMap(new IdentityHashMap<>()); + this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); + } + // Pool lock released — concurrent getAndLock() calls proceed immediately with fresh pool. + + // Step 2: Wait for in-flight operations on old items to complete. + // No pool lock held here, so no contention with concurrent callers. + List checkedOutItems = new ArrayList<>(oldItems.size()); + for (T item : oldItems) { + item.lock(); + try { + checkedOutItems.add(item); + } finally { + item.unlock(); } } return Collections.unmodifiableList(checkedOutItems); diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java index 75d7b37368f0c..5f095eec3eb07 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java @@ -13,6 +13,7 @@ import org.opensearch.index.engine.dataformat.DocumentInput; import org.opensearch.index.mapper.MappedFieldType; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -47,7 +48,7 @@ public CompositeDocumentInput( ) { this.primaryFormat = Objects.requireNonNull(primaryFormat, "primaryFormat must not be null"); this.primaryDocumentInput = Objects.requireNonNull(primaryDocumentInput, "primaryDocumentInput must not be null"); - this.secondaryDocumentInputs = Map.copyOf( + this.secondaryDocumentInputs = Collections.unmodifiableMap( Objects.requireNonNull(secondaryDocumentInputs, "secondaryDocumentInputs must not be null") ); } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java index b1080cbe9a63c..7f2d02a3d2b5d 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java @@ -31,7 +31,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.LinkedHashMap; +import java.util.IdentityHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -275,7 +275,7 @@ public void deleteFiles(Map> filesToDelete) throws IO @Override public CompositeDocumentInput newDocumentInput() { DocumentInput primaryInput = primaryEngine.newDocumentInput(); - Map> secondaryInputMap = new LinkedHashMap<>(); + Map> secondaryInputMap = new IdentityHashMap<>(); for (IndexingExecutionEngine engine : secondaryEngines) { secondaryInputMap.put(engine.getDataFormat(), engine.newDocumentInput()); } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java index 309494adf21e9..63c68dbbea0cd 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java @@ -21,8 +21,8 @@ import org.opensearch.index.engine.exec.WriterFileSet; import java.io.IOException; -import java.util.AbstractMap; -import java.util.LinkedHashMap; +import java.util.Collections; +import java.util.IdentityHashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -44,7 +44,8 @@ class CompositeWriter implements Writer, Lockable { private static final Logger logger = LogManager.getLogger(CompositeWriter.class); - private final Map.Entry>> primaryWriter; + private final DataFormat primaryFormat; + private final Writer> primaryWriter; private final Map>> secondaryWritersByFormat; private final ReentrantLock lock; private final long writerGeneration; @@ -87,16 +88,14 @@ enum WriterState { this.writerGeneration = writerGeneration; IndexingExecutionEngine primaryDelegate = engine.getPrimaryDelegate(); - this.primaryWriter = new AbstractMap.SimpleImmutableEntry<>( - primaryDelegate.getDataFormat(), - (Writer>) primaryDelegate.createWriter(writerGeneration) - ); + this.primaryFormat = primaryDelegate.getDataFormat(); + this.primaryWriter = (Writer>) primaryDelegate.createWriter(writerGeneration); - Map>> secondaries = new LinkedHashMap<>(); + Map>> secondaries = new IdentityHashMap<>(); for (IndexingExecutionEngine delegate : engine.getSecondaryDelegates()) { secondaries.put(delegate.getDataFormat(), (Writer>) delegate.createWriter(writerGeneration)); } - this.secondaryWritersByFormat = Map.copyOf(secondaries); + this.secondaryWritersByFormat = Collections.unmodifiableMap(secondaries); this.rowIdGenerator = new RowIdGenerator(CompositeWriter.class.getName()); } @@ -106,11 +105,11 @@ public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { throw new IllegalStateException("Cannot add document to writer in state " + state.get()); } // Write to primary first - WriteResult primaryResult = primaryWriter.getValue().addDoc(doc.getPrimaryInput()); + WriteResult primaryResult = primaryWriter.addDoc(doc.getPrimaryInput()); switch (primaryResult) { - case WriteResult.Success s -> logger.trace("Successfully added document in primary format [{}]", primaryWriter.getKey().name()); + case WriteResult.Success s -> logger.trace("Successfully added document in primary format [{}]", primaryFormat.name()); case WriteResult.Failure f -> { - logger.debug("Failed to add document in primary format [{}]", primaryWriter.getKey().name()); + logger.debug("Failed to add document in primary format [{}]", primaryFormat.name()); return primaryResult; } } @@ -141,8 +140,8 @@ public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { public FileInfos flush() throws IOException { FileInfos.Builder builder = FileInfos.builder(); // Flush primary - Optional primaryWfs = primaryWriter.getValue().flush().getWriterFileSet(primaryWriter.getKey()); - primaryWfs.ifPresent(writerFileSet -> builder.putWriterFileSet(primaryWriter.getKey(), writerFileSet)); + Optional primaryWfs = primaryWriter.flush().getWriterFileSet(primaryFormat); + primaryWfs.ifPresent(writerFileSet -> builder.putWriterFileSet(primaryFormat, writerFileSet)); // Flush secondaries for (Writer> writer : secondaryWritersByFormat.values()) { FileInfos fileInfos = writer.flush(); @@ -156,7 +155,7 @@ public FileInfos flush() throws IOException { @Override public void sync() throws IOException { - primaryWriter.getValue().sync(); + primaryWriter.sync(); for (Writer> writer : secondaryWritersByFormat.values()) { writer.sync(); } @@ -164,7 +163,7 @@ public void sync() throws IOException { @Override public void close() throws IOException { - primaryWriter.getValue().close(); + primaryWriter.close(); for (Writer> writer : secondaryWritersByFormat.values()) { writer.close(); } From e93d27a90223a01045690afb733d09c4435c0912 Mon Sep 17 00:00:00 2001 From: Arpit Bandejiya Date: Sun, 29 Mar 2026 00:20:54 +0530 Subject: [PATCH 7/9] Simplify locks and use IdentifyHashMap Signed-off-by: Bukhtawar Khan Co-authored-by: Arpit Bandejiya --- .../common/queue/LockablePoolBenchmark.java | 159 ++++++++++++++++++ .../opensearch/common/queue/LockablePool.java | 53 +++--- .../composite/CompositeDocumentInput.java | 3 +- .../CompositeIndexingExecutionEngine.java | 4 +- .../opensearch/composite/CompositeWriter.java | 31 ++-- 5 files changed, 210 insertions(+), 40 deletions(-) create mode 100644 libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java diff --git a/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java new file mode 100644 index 0000000000000..22c4163d00116 --- /dev/null +++ b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java @@ -0,0 +1,159 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.queue; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Group; +import org.openjdk.jmh.annotations.GroupThreads; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; + +/** + * JMH benchmark for {@link LockablePool} measuring: + *
    + *
  • Isolated checkout/return throughput at varying thread counts
  • + *
  • Mixed workload: concurrent writers + periodic checkoutAll (refresh)
  • + *
+ * The mixed group benchmark is the most realistic — it models the composite + * engine's write path where indexing threads hold writers while a refresh + * thread periodically drains the pool via checkoutAll. + */ +@Fork(2) +@Warmup(iterations = 2, time = 3) +@Measurement(iterations = 3, time = 5) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@State(Scope.Benchmark) +@SuppressWarnings("unused") +public class LockablePoolBenchmark { + + @Param({ "4", "8" }) + int concurrency; + + private LockablePool pool; + + @Setup(Level.Iteration) + public void setup() { + AtomicInteger counter = new AtomicInteger(0); + pool = new LockablePool<>(() -> new PoolEntry(counter.getAndIncrement()), LinkedList::new, concurrency); + // Pre-warm the pool + for (int i = 0; i < concurrency * 2; i++) { + PoolEntry e = pool.getAndLock(); + pool.releaseAndUnlock(e); + } + } + + // ---- Mixed workload: writers + periodic refresh (checkoutAll) ---- + // This is the realistic scenario: multiple indexing threads hold writers + // while a single refresh thread periodically drains the pool. + + @Benchmark + @Group("mixed_7w_1r") + @GroupThreads(7) + public void writers_7w1r(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("mixed_7w_1r") + @GroupThreads(1) + public List refresh_7w1r() throws InterruptedException { + Thread.sleep(1000); // simulate 1s refresh interval + return pool.checkoutAll(); + } + + @Benchmark + @Group("mixed_3w_1r") + @GroupThreads(3) + public void writers_3w1r(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("mixed_3w_1r") + @GroupThreads(1) + public List refresh_3w1r() throws InterruptedException { + Thread.sleep(1000); // simulate 1s refresh interval + return pool.checkoutAll(); + } + + // ---- Isolated: pure writer throughput (no refresh contention) ---- + + @Benchmark + @Group("writers_only_4t") + @GroupThreads(4) + public void writersOnly_4t(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("writers_only_8t") + @GroupThreads(8) + public void writersOnly_8t(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + private static long simulateWork(PoolEntry entry) { + long result = entry.hashCode(); + for (int i = 0; i < 20; i++) { + result ^= (result << 13); + result ^= (result >> 7); + result ^= (result << 17); + } + return result; + } + + static final class PoolEntry implements Lockable { + final int id; + private final ReentrantLock lock = new ReentrantLock(); + + PoolEntry(int id) { + this.id = id; + } + + @Override + public void lock() { + lock.lock(); + } + + @Override + public boolean tryLock() { + return lock.tryLock(); + } + + @Override + public void unlock() { + lock.unlock(); + } + } +} diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java index 505c92abd5833..e1e8a714f9ded 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java @@ -31,9 +31,11 @@ */ public final class LockablePool implements Iterable, Closeable { - private final Set items; - private final LockableConcurrentQueue availableItems; + private volatile Set items; + private volatile LockableConcurrentQueue availableItems; private final Supplier itemSupplier; + private final Supplier> queueSupplier; + private final int concurrency; private volatile boolean closed; /** @@ -46,6 +48,8 @@ public final class LockablePool implements Iterable, Clos public LockablePool(Supplier itemSupplier, Supplier> queueSupplier, int concurrency) { this.items = Collections.newSetFromMap(new IdentityHashMap<>()); this.itemSupplier = Objects.requireNonNull(itemSupplier, "itemSupplier must not be null"); + this.queueSupplier = Objects.requireNonNull(queueSupplier, "queueSupplier must not be null"); + this.concurrency = concurrency; this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); } @@ -70,39 +74,46 @@ private synchronized T fetchItem() { } /** - * Releases the given item back to this pool for reuse. + * Releases the given item back to this pool for reuse. If the item belongs to a previous + * generation (swapped out during {@link #checkoutAll()}), it is silently unlocked and + * discarded since the checkout caller owns it. * * @param item the item to release */ public void releaseAndUnlock(T item) { - assert isRegistered(item) : "Pool doesn't know about this item"; + if (isRegistered(item) == false) { + item.unlock(); + return; + } availableItems.addAndUnlock(item); } /** - * Lock and checkout all items from the pool. + * Atomically swaps the pool's item set and queue with fresh instances, then waits for + * any in-flight operations on the old items to complete. This minimizes the time the pool + * lock is held — callers of {@link #getAndLock()} see the new empty pool immediately and + * can create fresh items without waiting for the checkout to finish. * - * @return unmodifiable list of all items locked by current thread + * @return unmodifiable list of all checked-out items * @throws IllegalStateException if the pool is closed */ public List checkoutAll() { ensureOpen(); - List lockedItems = new ArrayList<>(); - List checkedOutItems = new ArrayList<>(); - for (T item : this) { - item.lock(); - lockedItems.add(item); - } + + Set oldItems; synchronized (this) { - for (T item : lockedItems) { - try { - if (isRegistered(item) && items.remove(item)) { - availableItems.remove(item); - checkedOutItems.add(item); - } - } finally { - item.unlock(); - } + oldItems = this.items; + this.items = Collections.newSetFromMap(new IdentityHashMap<>()); + this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); + } + + List checkedOutItems = new ArrayList<>(oldItems.size()); + for (T item : oldItems) { + item.lock(); + try { + checkedOutItems.add(item); + } finally { + item.unlock(); } } return Collections.unmodifiableList(checkedOutItems); diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java index 75d7b37368f0c..5f095eec3eb07 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeDocumentInput.java @@ -13,6 +13,7 @@ import org.opensearch.index.engine.dataformat.DocumentInput; import org.opensearch.index.mapper.MappedFieldType; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -47,7 +48,7 @@ public CompositeDocumentInput( ) { this.primaryFormat = Objects.requireNonNull(primaryFormat, "primaryFormat must not be null"); this.primaryDocumentInput = Objects.requireNonNull(primaryDocumentInput, "primaryDocumentInput must not be null"); - this.secondaryDocumentInputs = Map.copyOf( + this.secondaryDocumentInputs = Collections.unmodifiableMap( Objects.requireNonNull(secondaryDocumentInputs, "secondaryDocumentInputs must not be null") ); } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java index b1080cbe9a63c..7f2d02a3d2b5d 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeIndexingExecutionEngine.java @@ -31,7 +31,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.LinkedHashMap; +import java.util.IdentityHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -275,7 +275,7 @@ public void deleteFiles(Map> filesToDelete) throws IO @Override public CompositeDocumentInput newDocumentInput() { DocumentInput primaryInput = primaryEngine.newDocumentInput(); - Map> secondaryInputMap = new LinkedHashMap<>(); + Map> secondaryInputMap = new IdentityHashMap<>(); for (IndexingExecutionEngine engine : secondaryEngines) { secondaryInputMap.put(engine.getDataFormat(), engine.newDocumentInput()); } diff --git a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java index 309494adf21e9..63c68dbbea0cd 100644 --- a/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java +++ b/sandbox/plugins/composite-engine/src/main/java/org/opensearch/composite/CompositeWriter.java @@ -21,8 +21,8 @@ import org.opensearch.index.engine.exec.WriterFileSet; import java.io.IOException; -import java.util.AbstractMap; -import java.util.LinkedHashMap; +import java.util.Collections; +import java.util.IdentityHashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -44,7 +44,8 @@ class CompositeWriter implements Writer, Lockable { private static final Logger logger = LogManager.getLogger(CompositeWriter.class); - private final Map.Entry>> primaryWriter; + private final DataFormat primaryFormat; + private final Writer> primaryWriter; private final Map>> secondaryWritersByFormat; private final ReentrantLock lock; private final long writerGeneration; @@ -87,16 +88,14 @@ enum WriterState { this.writerGeneration = writerGeneration; IndexingExecutionEngine primaryDelegate = engine.getPrimaryDelegate(); - this.primaryWriter = new AbstractMap.SimpleImmutableEntry<>( - primaryDelegate.getDataFormat(), - (Writer>) primaryDelegate.createWriter(writerGeneration) - ); + this.primaryFormat = primaryDelegate.getDataFormat(); + this.primaryWriter = (Writer>) primaryDelegate.createWriter(writerGeneration); - Map>> secondaries = new LinkedHashMap<>(); + Map>> secondaries = new IdentityHashMap<>(); for (IndexingExecutionEngine delegate : engine.getSecondaryDelegates()) { secondaries.put(delegate.getDataFormat(), (Writer>) delegate.createWriter(writerGeneration)); } - this.secondaryWritersByFormat = Map.copyOf(secondaries); + this.secondaryWritersByFormat = Collections.unmodifiableMap(secondaries); this.rowIdGenerator = new RowIdGenerator(CompositeWriter.class.getName()); } @@ -106,11 +105,11 @@ public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { throw new IllegalStateException("Cannot add document to writer in state " + state.get()); } // Write to primary first - WriteResult primaryResult = primaryWriter.getValue().addDoc(doc.getPrimaryInput()); + WriteResult primaryResult = primaryWriter.addDoc(doc.getPrimaryInput()); switch (primaryResult) { - case WriteResult.Success s -> logger.trace("Successfully added document in primary format [{}]", primaryWriter.getKey().name()); + case WriteResult.Success s -> logger.trace("Successfully added document in primary format [{}]", primaryFormat.name()); case WriteResult.Failure f -> { - logger.debug("Failed to add document in primary format [{}]", primaryWriter.getKey().name()); + logger.debug("Failed to add document in primary format [{}]", primaryFormat.name()); return primaryResult; } } @@ -141,8 +140,8 @@ public WriteResult addDoc(CompositeDocumentInput doc) throws IOException { public FileInfos flush() throws IOException { FileInfos.Builder builder = FileInfos.builder(); // Flush primary - Optional primaryWfs = primaryWriter.getValue().flush().getWriterFileSet(primaryWriter.getKey()); - primaryWfs.ifPresent(writerFileSet -> builder.putWriterFileSet(primaryWriter.getKey(), writerFileSet)); + Optional primaryWfs = primaryWriter.flush().getWriterFileSet(primaryFormat); + primaryWfs.ifPresent(writerFileSet -> builder.putWriterFileSet(primaryFormat, writerFileSet)); // Flush secondaries for (Writer> writer : secondaryWritersByFormat.values()) { FileInfos fileInfos = writer.flush(); @@ -156,7 +155,7 @@ public FileInfos flush() throws IOException { @Override public void sync() throws IOException { - primaryWriter.getValue().sync(); + primaryWriter.sync(); for (Writer> writer : secondaryWritersByFormat.values()) { writer.sync(); } @@ -164,7 +163,7 @@ public void sync() throws IOException { @Override public void close() throws IOException { - primaryWriter.getValue().close(); + primaryWriter.close(); for (Writer> writer : secondaryWritersByFormat.values()) { writer.close(); } From 530d346063db63282aee638891fb08a4c6030132 Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sun, 29 Mar 2026 14:13:14 +0530 Subject: [PATCH 8/9] Simplify the locks during refresh, use IdentifyHashMap Signed-off-by: Bukhtawar Khan --- .../common/queue/ConcurrentQueue.java | 20 +++++ .../common/queue/LockableConcurrentQueue.java | 10 +++ .../opensearch/common/queue/LockablePool.java | 74 ++++++++++--------- 3 files changed, 70 insertions(+), 34 deletions(-) diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java index f7856c9464f54..b37d73eabc333 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/ConcurrentQueue.java @@ -137,4 +137,24 @@ boolean remove(T entry) { } return false; } + + /** + * Removes all entries matching the given predicate in a single pass across all stripes. + * Each stripe is locked once, and all matching entries within that stripe are removed + * before moving to the next stripe. + * + * @param predicate the condition for removal + */ + void removeIf(Predicate predicate) { + for (int i = 0; i < concurrency; ++i) { + final Lock lock = locks[i]; + final Queue queue = queues[i]; + lock.lock(); + try { + queue.removeIf(predicate); + } finally { + lock.unlock(); + } + } + } } diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java index 8d03a73fde08e..75005b4a88f99 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockableConcurrentQueue.java @@ -10,6 +10,7 @@ import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; import java.util.function.Supplier; /** @@ -66,6 +67,15 @@ public boolean remove(T entry) { return queue.remove(entry); } + /** + * Removes all entries matching the given predicate in a single pass across all stripes. + * + * @param predicate the condition for removal + */ + public void removeIf(Predicate predicate) { + queue.removeIf(predicate); + } + /** * Add an entry to the queue and unlock it, in that order. * diff --git a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java index d11b7dec9ff4e..ddc24b364c46e 100644 --- a/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java +++ b/libs/concurrent-queue/src/main/java/org/opensearch/common/queue/LockablePool.java @@ -31,11 +31,9 @@ */ public final class LockablePool implements Iterable, Closeable { - private volatile Set items; - private volatile LockableConcurrentQueue availableItems; + private final Set items; + private final LockableConcurrentQueue availableItems; private final Supplier itemSupplier; - private final Supplier> queueSupplier; - private final int concurrency; private volatile boolean closed; /** @@ -48,8 +46,6 @@ public final class LockablePool implements Iterable, Clos public LockablePool(Supplier itemSupplier, Supplier> queueSupplier, int concurrency) { this.items = Collections.newSetFromMap(new IdentityHashMap<>()); this.itemSupplier = Objects.requireNonNull(itemSupplier, "itemSupplier must not be null"); - this.queueSupplier = Objects.requireNonNull(queueSupplier, "queueSupplier must not be null"); - this.concurrency = concurrency; this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); } @@ -74,54 +70,64 @@ private synchronized T fetchItem() { } /** - * Releases the given item back to this pool for reuse. If the item belongs to a previous - * generation (swapped out during {@link #checkoutAll()}), it is silently unlocked and - * discarded since the checkout caller owns it. + * Releases the given item back to this pool for reuse. * * @param item the item to release */ public void releaseAndUnlock(T item) { - if (isRegistered(item) == false) { - // Item belongs to a previous generation swapped out during checkoutAll(). - // Just unlock it — the checkout caller owns it now. - item.unlock(); - return; - } + assert isRegistered(item) : "Pool doesn't know about this item"; availableItems.addAndUnlock(item); } /** - * Atomically swaps the pool's item set and queue with fresh instances, then waits for - * any in-flight operations on the old items to complete. This minimizes the time the pool - * lock is held — callers of {@link #getAndLock()} see the new empty pool immediately and - * can create fresh items without waiting for the checkout to finish. + * Lock and checkout all items from the pool. + *

+ * Phase 1: Snapshot the items set under the pool lock. + * Phase 2: Lock each item outside the monitor to avoid holding it while blocking on in-flight operations. + * Phase 3: Remove checked-out items from the set and bulk-remove from the queue in a single pass. * - * @return unmodifiable list of all checked-out items + * @return unmodifiable list of all items locked by current thread * @throws IllegalStateException if the pool is closed */ public List checkoutAll() { ensureOpen(); - // Step 1: Atomic swap — hold pool lock only for the reference swap. - Set oldItems; + // Phase 1: Snapshot + List snapshot; synchronized (this) { - oldItems = this.items; - this.items = Collections.newSetFromMap(new IdentityHashMap<>()); - this.availableItems = new LockableConcurrentQueue<>(queueSupplier, concurrency); + if (items.isEmpty()) { + return Collections.emptyList(); + } + snapshot = new ArrayList<>(items.size()); + snapshot.addAll(items); } - // Pool lock released — concurrent getAndLock() calls proceed immediately with fresh pool. - // Step 2: Wait for in-flight operations on old items to complete. - // No pool lock held here, so no contention with concurrent callers. - List checkedOutItems = new ArrayList<>(oldItems.size()); - for (T item : oldItems) { + // Phase 2: Lock outside monitor + for (T item : snapshot) { item.lock(); - try { - checkedOutItems.add(item); - } finally { - item.unlock(); + } + + // Phase 3: Process + bulk cleanup + List checkedOutItems = new ArrayList<>(snapshot.size()); + synchronized (this) { + Set toRemoveFromQueue = Collections.newSetFromMap(new IdentityHashMap<>(snapshot.size())); + + for (T item : snapshot) { + try { + if (items.remove(item)) { + toRemoveFromQueue.add(item); + checkedOutItems.add(item); + } + } finally { + item.unlock(); + } + } + + if (toRemoveFromQueue.isEmpty() == false) { + availableItems.removeIf(toRemoveFromQueue::contains); } } + return Collections.unmodifiableList(checkedOutItems); } From 30aea8f75c486280301197a1e52f11eeb9c007ab Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Sun, 29 Mar 2026 16:28:49 +0530 Subject: [PATCH 9/9] Improve benchmark Signed-off-by: Bukhtawar Khan --- .../common/queue/LockablePoolBenchmark.java | 120 ++++++++++++++++-- 1 file changed, 107 insertions(+), 13 deletions(-) diff --git a/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java index 22c4163d00116..38f2df3f5fa18 100644 --- a/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java +++ b/libs/concurrent-queue/src/jmh/java/org/opensearch/common/queue/LockablePoolBenchmark.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.LockSupport; import java.util.concurrent.locks.ReentrantLock; /** @@ -35,16 +36,13 @@ *

    *
  • Isolated checkout/return throughput at varying thread counts
  • *
  • Mixed workload: concurrent writers + periodic checkoutAll (refresh)
  • + *
  • Aggressive refresh: frequent checkoutAll to stress Phase 3 bulk cleanup
  • + *
  • Writer latency during refresh contention (sample mode)
  • *
- * The mixed group benchmark is the most realistic — it models the composite - * engine's write path where indexing threads hold writers while a refresh - * thread periodically drains the pool via checkoutAll. */ @Fork(2) @Warmup(iterations = 2, time = 3) @Measurement(iterations = 3, time = 5) -@BenchmarkMode(Mode.Throughput) -@OutputTimeUnit(TimeUnit.MICROSECONDS) @State(Scope.Benchmark) @SuppressWarnings("unused") public class LockablePoolBenchmark { @@ -58,20 +56,20 @@ public class LockablePoolBenchmark { public void setup() { AtomicInteger counter = new AtomicInteger(0); pool = new LockablePool<>(() -> new PoolEntry(counter.getAndIncrement()), LinkedList::new, concurrency); - // Pre-warm the pool - for (int i = 0; i < concurrency * 2; i++) { + // Pre-warm the pool with more entries to populate multiple stripes + for (int i = 0; i < concurrency * 4; i++) { PoolEntry e = pool.getAndLock(); pool.releaseAndUnlock(e); } } - // ---- Mixed workload: writers + periodic refresh (checkoutAll) ---- - // This is the realistic scenario: multiple indexing threads hold writers - // while a single refresh thread periodically drains the pool. + // ── Mixed workload: writers + periodic refresh (1s interval) ── @Benchmark @Group("mixed_7w_1r") @GroupThreads(7) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public void writers_7w1r(Blackhole bh) { PoolEntry e = pool.getAndLock(); bh.consume(simulateWork(e)); @@ -81,14 +79,18 @@ public void writers_7w1r(Blackhole bh) { @Benchmark @Group("mixed_7w_1r") @GroupThreads(1) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public List refresh_7w1r() throws InterruptedException { - Thread.sleep(1000); // simulate 1s refresh interval + Thread.sleep(1000); return pool.checkoutAll(); } @Benchmark @Group("mixed_3w_1r") @GroupThreads(3) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public void writers_3w1r(Blackhole bh) { PoolEntry e = pool.getAndLock(); bh.consume(simulateWork(e)); @@ -98,16 +100,91 @@ public void writers_3w1r(Blackhole bh) { @Benchmark @Group("mixed_3w_1r") @GroupThreads(1) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public List refresh_3w1r() throws InterruptedException { - Thread.sleep(1000); // simulate 1s refresh interval + Thread.sleep(1000); return pool.checkoutAll(); } - // ---- Isolated: pure writer throughput (no refresh contention) ---- + // ── Aggressive refresh: 10ms interval to stress checkoutAll Phase 3 ── + // This makes checkoutAll fire ~100x/sec instead of 1x/sec, amplifying + // the difference between per-item remove (old) and bulk removeIf (new). + + @Benchmark + @Group("aggressive_7w_1r") + @GroupThreads(7) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void writers_aggressive_7w1r(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("aggressive_7w_1r") + @GroupThreads(1) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public List refresh_aggressive_7w1r() { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10)); + return pool.checkoutAll(); + } + + @Benchmark + @Group("aggressive_3w_1r") + @GroupThreads(3) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void writers_aggressive_3w1r(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("aggressive_3w_1r") + @GroupThreads(1) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public List refresh_aggressive_3w1r() { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10)); + return pool.checkoutAll(); + } + + // ── Writer latency during refresh contention (sample mode) ── + // Measures per-operation latency distribution to capture tail latency + // spikes caused by checkoutAll holding the pool lock. + + @Benchmark + @Group("latency_7w_1r") + @GroupThreads(7) + @BenchmarkMode(Mode.SampleTime) + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public void writers_latency_7w1r(Blackhole bh) { + PoolEntry e = pool.getAndLock(); + bh.consume(simulateWork(e)); + pool.releaseAndUnlock(e); + } + + @Benchmark + @Group("latency_7w_1r") + @GroupThreads(1) + @BenchmarkMode(Mode.SampleTime) + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public List refresh_latency_7w1r() { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10)); + return pool.checkoutAll(); + } + + // ── Isolated: pure writer throughput (no refresh contention) ── @Benchmark @Group("writers_only_4t") @GroupThreads(4) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public void writersOnly_4t(Blackhole bh) { PoolEntry e = pool.getAndLock(); bh.consume(simulateWork(e)); @@ -117,12 +194,29 @@ public void writersOnly_4t(Blackhole bh) { @Benchmark @Group("writers_only_8t") @GroupThreads(8) + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.MICROSECONDS) public void writersOnly_8t(Blackhole bh) { PoolEntry e = pool.getAndLock(); bh.consume(simulateWork(e)); pool.releaseAndUnlock(e); } + // ── Isolated checkoutAll throughput (no writers) ── + // Directly measures checkoutAll cost with a pre-populated pool. + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public List checkoutAll_isolated() { + // Re-populate pool before each checkout + for (int i = 0; i < concurrency; i++) { + PoolEntry e = pool.getAndLock(); + pool.releaseAndUnlock(e); + } + return pool.checkoutAll(); + } + private static long simulateWork(PoolEntry entry) { long result = entry.hashCode(); for (int i = 0; i < 20; i++) {