vigyasharma commented on code in PR #14900:
URL: https://github.com/apache/lucene/pull/14900#discussion_r2205707984


##########
lucene/core/src/java/org/apache/lucene/index/SharedMergeScheduler.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.lucene.index;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * SharedMergeScheduler is an experimental MergeScheduler that submits merge 
tasks to a shared
+ * thread pool across IndexWriters.
+ */
+public class SharedMergeScheduler extends MergeScheduler {
+
+ /**
+ * Executor service provided externally to handle merge tasks.
+ * Allows sharing a thread pool across IndexWriters if configured that way.
+ */
+  private final ExecutorService sharedExecutor;
+
+  public SharedMergeScheduler(ExecutorService sharedExecutor) {
+      this.sharedExecutor = sharedExecutor;
+  }
+
+  /**
+   * Retrieves pending merge tasks from the given {@link MergeSource} and 
submits them to the shared
+   * thread pool for execution.
+   *
+   * @param mergeSource the source of merge tasks (typically an IndexWriter)
+   * @param trigger the event that triggered the merge (e.g., SEGMENT_FLUSH, 
EXPLICIT)
+   * @throws IOException if merging fails
+   */
+  @Override
+  public void merge(MergeSource mergeSource, MergeTrigger trigger) throws 
IOException {
+    while (true) {
+      final MergePolicy.OneMerge merge = mergeSource.getNextMerge();
+      if (merge == null) break;
+
+      Runnable mergeRunnable = () -> {
+          try {
+              mergeSource.merge(merge);
+          } catch (IOException e) {
+              throw new RuntimeException(e);
+          }
+      };
+
+      MergeTaskWrapper wrappedTask = new MergeTaskWrapper(mergeRunnable, 
mergeSource, merge.totalBytesSize());
+      sharedExecutor.submit(wrappedTask.getMergeTask());
+          }
+  }
+
+  /**
+   * Closes the merge scheduler. This implementation is currently a no-op. In 
a production-ready
+   * version, the shared executor should be properly shut down.
+   */
+  @Override
+  public void close() {

Review Comment:
   This impl. for `close()` will shut down the entire executor. Since this is a 
shared merge scheduler, you only want to address merges related to the closing 
IndexWriter, like cancelling all pending merges and waiting for running merges 
to complete. (See[ 
close()](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java#L470)
 in CMS as well).



##########
lucene/core/src/java/org/apache/lucene/index/MergeTaskWrapper.java:
##########
@@ -0,0 +1,25 @@
+package org.apache.lucene.index;
+
+public class MergeTaskWrapper {
+    private final Runnable mergeTask;
+    private final Object sourceWriter; // Can be IndexWriter or its ID
+    private final long mergeSize;      // For priority-based scheduling
+
+    public MergeTaskWrapper(Runnable mergeTask, Object sourceWriter, long 
mergeSize) {
+        this.mergeTask = mergeTask;
+        this.sourceWriter = sourceWriter;
+        this.mergeSize = mergeSize;
+    }
+
+    public Runnable getMergeTask() {
+        return mergeTask;
+    }
+
+    public Object getSourceWriter() {
+        return sourceWriter;
+    }
+
+    public long getMergeSize() {
+        return mergeSize;
+    }
+}

Review Comment:
   nit: we keep an empty line at the end of file. There might be an 
editor/formatter setting you want to add that brings it in automatically.



##########
lucene/core/src/java/org/apache/lucene/index/MergeTaskWrapper.java:
##########
@@ -0,0 +1,25 @@
+package org.apache.lucene.index;
+
+public class MergeTaskWrapper {
+    private final Runnable mergeTask;
+    private final Object sourceWriter; // Can be IndexWriter or its ID

Review Comment:
   You can directly use `IndexWriter` or `String` for ID instead of using 
Object. Keeping the type as specific as possible helps catch a lot of bugs 
before runtime.



##########
lucene/core/src/java/org/apache/lucene/index/MergeTaskWrapper.java:
##########
@@ -0,0 +1,25 @@
+package org.apache.lucene.index;
+
+public class MergeTaskWrapper {

Review Comment:
   I'm curious how we intend to use this wrapper, looking forward to the next 
iteration of this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org

Reply via email to