You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/02/01 07:41:14 UTC

[GitHub] [lucene] vigyasharma opened a new pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

vigyasharma opened a new pull request #633:
URL: https://github.com/apache/lucene/pull/633


   <!--
   _(If you are a project committer then you may remove some/all of the following template.)_
   
   Before creating a pull request, please file an issue in the ASF Jira system for Lucene:
   
   * https://issues.apache.org/jira/projects/LUCENE
   
   You will need to create an account in Jira in order to create an issue.
   
   The title of the PR should reference the Jira issue number in the form:
   
   * LUCENE-####: <short description of problem or changes>
   
   LUCENE must be fully capitalized. A short description helps people scanning pull requests for items they can work on.
   
   Properly referencing the issue in the title ensures that Jira is correctly updated with code review comments and commits. -->
   
   
   # Description
   
   > _Work in Progress PR to share approach and get early feedback on changes_
   
   The addIndexes(CodecReader...) API today merges all provided readers into a single merge, in one large blocking call. We want to add concurrency here by invoking multiple parallel merges on subsets of readers, in a way that is configurable by users. The merged segments created, can later be merged further in the regular, non-blocking, background merges triggered by Lucene. Currently, users are responsible  for managing their API run times, by invoking it multiple times with subsets of readers. 
   JIRA - https://issues.apache.org/jira/browse/LUCENE-10216
   
   
   # Solution
   
   In this change, we leverage the configured MergeScheduler to invoke all merges required by the addIndexes API. MergePolicy also exposes a `findMerges(List<CodecReader> readers)` API that users can override to configure how their readers should be clustered into merges. We add a default implementation that retains current behavior of adding all readers into a single merge.
   
   
   # Tests
   
   - Existing tests passing
   - Pending: New tests to be added
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [ ] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [ ] I have created a Jira issue and added the issue ID to my pull request title.
   - [ ] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [ ] I have developed this patch against the `main` branch.
   - [ ] I have run `./gradlew check`.
   - [ ] I have added tests for my changes.
   


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822174425



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -352,6 +352,14 @@ public FieldDimensions(int dimensionCount, int indexDimensionCount, int dimensio
       this.softDeletesFieldName = softDeletesFieldName;
     }
 
+    public void verifyFieldInfo(FieldInfo fi) {

Review comment:
       This method makes sure that incoming index fields are compatible with the destination index, e.g. vectors have the same dimensions and use the same similarity function.
   
   Changed access to package private. I retained method name - `verifyFieldInfo()`, as it internally calls `verifySoftDeletedFieldName` and `verifySameSchema`, which have a similar naming style. 
   
   I can change it if `assert*()` is more in line with the Lucene convention. In which case, I also think that it should only return a condition and the asserts should be done by the caller.. Let me know..




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822174617



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:

Review comment:
       I think it was a miss. Added it back, thanks!




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822189054



##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -567,6 +605,21 @@ public abstract MergeSpecification findMerges(
       MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
       throws IOException;
 
+  /**
+   * Define {@link OneMerge} operations for a list of codec readers. This call is used to
+   * define merges for input readers in {@link IndexWriter#addIndexes(CodecReader...)}.
+   * Default implementation adds all readers to a single merge. This can be overridden in custom
+   * merge policies.
+   *
+   * @param readers set of readers to merge into the main index
+   */
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    OneMerge merge = new OneMerge(readers, leaf -> new MergeReader(leaf, leaf.getLiveDocs()));

Review comment:
       Yes, this change will modify addIndexes to leverage `MergePolicy` and `MergeScheduler`, without changing its end behavior. 
   
   I am thinking of doing a follow up PR with a merge policy that does a 1:1 addIndexes call, where the merger creates one segment each for every provided reader. It would create a faster, more concurrent `addIndexes(CodecReader...)`, at the cost of deferring some merges to be done later in background. 
   
   Which, I believe is similar to the behavior in `addIndexes(Directory...)` - all incoming segments are simply added to IW, and any merging happens in later in background.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] mikemccand commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r819703946



##########
File path: lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
##########
@@ -352,6 +352,14 @@ public FieldDimensions(int dimensionCount, int indexDimensionCount, int dimensio
       this.softDeletesFieldName = softDeletesFieldName;
     }
 
+    public void verifyFieldInfo(FieldInfo fi) {

Review comment:
       Should this really be public?  Or is it only for `asserting`, in which case maybe make it package private and rename to `boolean assertFieldInfo` or so?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:

Review comment:
       Hmm we lost this comment?  Was that intentional?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +

Review comment:
       I think this only happens if IW is aborting/closing?  Maybe throw `AlreadyClosedException` instead?  This message is otherwise sort of confusing to the poor user who sees it.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));

Review comment:
       Normal merges have the opportunity to "cascade": `MergePolicy` picks a round of initial merges, and as they complete and get swapped in, it is consulted again later and might choose to merge some of those just-merged segments into even bigger ones.  But it looks like we don't allow that here?  We ask once, do those merges, commit to IW's `SegmentInfos`?  Maybe that's fine, it's just different.

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {
+            // nocommit -- ensure all intermediate files are deleted
+            for (MergePolicy.OneMerge merge: spec.merges) {
+              deleteNewFiles(merge.getMergeInfo().files());
+            }
+          }
+        }
+      }
 
-      // TODO: somehow we should fix this merge so it's
-      // abortable so that IW.close(false) is able to stop it
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-      Codec codec = config.getCodec();
-      // We set the min version to null for now, it will be set later by SegmentMerger
-      SegmentInfo info =
-          new SegmentInfo(
-              directoryOrig,
-              Version.LATEST,
-              null,
-              mergedName,
-              -1,
-              false,
-              codec,
-              Collections.emptyMap(),
-              StringHelper.randomId(),
-              Collections.emptyMap(),
-              config.getIndexSort());
-
-      SegmentMerger merger =
-          new SegmentMerger(
-              Arrays.asList(readers), info, infoStream, trackingDir, globalFieldNumberMap, context);
+      if (mergesComplete) {
+        List<SegmentCommitInfo> infos = new ArrayList<>();
+        long totalMaxDoc = 0;
+        for (MergePolicy.OneMerge merge: spec.merges) {
+          totalMaxDoc += merge.totalMaxDoc;
+          if (merge.getMergeInfo() != null) {

Review comment:
       Would this only be `null` if all documents in the merged segment had been deleted?  Maybe add a comment?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {

Review comment:
       `== false` instead?

##########
File path: lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
##########
@@ -39,6 +40,11 @@ public MergeSpecification findMerges(
     return null;
   }
 
+  @Override
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    return null;

Review comment:
       Hmm should `null` even be allowed by this new API?  Won't this throw `NPE` if someone tries to `addIndexes`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {
+            // nocommit -- ensure all intermediate files are deleted
+            for (MergePolicy.OneMerge merge: spec.merges) {
+              deleteNewFiles(merge.getMergeInfo().files());
+            }
+          }
+        }
+      }
 
-      // TODO: somehow we should fix this merge so it's
-      // abortable so that IW.close(false) is able to stop it
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-      Codec codec = config.getCodec();
-      // We set the min version to null for now, it will be set later by SegmentMerger
-      SegmentInfo info =
-          new SegmentInfo(
-              directoryOrig,
-              Version.LATEST,
-              null,
-              mergedName,
-              -1,
-              false,
-              codec,
-              Collections.emptyMap(),
-              StringHelper.randomId(),
-              Collections.emptyMap(),
-              config.getIndexSort());
-
-      SegmentMerger merger =
-          new SegmentMerger(
-              Arrays.asList(readers), info, infoStream, trackingDir, globalFieldNumberMap, context);
+      if (mergesComplete) {
+        List<SegmentCommitInfo> infos = new ArrayList<>();
+        long totalMaxDoc = 0;
+        for (MergePolicy.OneMerge merge: spec.merges) {
+          totalMaxDoc += merge.totalMaxDoc;
+          if (merge.getMergeInfo() != null) {
+            infos.add(merge.getMergeInfo());
+          }
+        }
 
-      if (!merger.shouldMerge()) {
-        return docWriter.getNextSequenceNumber();
+        // nocommit -- add tests for this transactional behavior
+        synchronized (this) {
+          if (infos.isEmpty() == false) {
+            boolean success = false;
+            try {
+              ensureOpen();
+              // Reserve the docs, just before we update SIS:
+              reserveDocs(totalMaxDoc);
+              success = true;
+            } finally {
+              if (!success) {

Review comment:
       `== false`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -567,6 +605,21 @@ public abstract MergeSpecification findMerges(
       MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
       throws IOException;
 
+  /**
+   * Define {@link OneMerge} operations for a list of codec readers. This call is used to
+   * define merges for input readers in {@link IndexWriter#addIndexes(CodecReader...)}.
+   * Default implementation adds all readers to a single merge. This can be overridden in custom
+   * merge policies.
+   *
+   * @param readers set of readers to merge into the main index
+   */
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    OneMerge merge = new OneMerge(readers, leaf -> new MergeReader(leaf, leaf.getLiveDocs()));

Review comment:
       So this means `MergePolicy` by default matches the behavior `IW.addIndexes(CodecReader[])` today -- doing a single large merge.  Good!

##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -813,12 +866,24 @@ protected final boolean verbose(MergeContext mergeContext) {
   }
 
   static final class MergeReader {
+    final CodecReader codecReader;
     final SegmentReader reader;

Review comment:
       Hmm it's a little spooky/confusing/smelly having both of these -- does anything still require accessing via `SegmentReader`?  Can they switch to `CodecReader`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {
+            // nocommit -- ensure all intermediate files are deleted
+            for (MergePolicy.OneMerge merge: spec.merges) {
+              deleteNewFiles(merge.getMergeInfo().files());
+            }
+          }
+        }
+      }
 
-      // TODO: somehow we should fix this merge so it's
-      // abortable so that IW.close(false) is able to stop it
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-      Codec codec = config.getCodec();
-      // We set the min version to null for now, it will be set later by SegmentMerger
-      SegmentInfo info =
-          new SegmentInfo(
-              directoryOrig,
-              Version.LATEST,
-              null,
-              mergedName,
-              -1,
-              false,
-              codec,
-              Collections.emptyMap(),
-              StringHelper.randomId(),
-              Collections.emptyMap(),
-              config.getIndexSort());
-
-      SegmentMerger merger =
-          new SegmentMerger(
-              Arrays.asList(readers), info, infoStream, trackingDir, globalFieldNumberMap, context);
+      if (mergesComplete) {
+        List<SegmentCommitInfo> infos = new ArrayList<>();
+        long totalMaxDoc = 0;
+        for (MergePolicy.OneMerge merge: spec.merges) {
+          totalMaxDoc += merge.totalMaxDoc;
+          if (merge.getMergeInfo() != null) {
+            infos.add(merge.getMergeInfo());
+          }
+        }
 
-      if (!merger.shouldMerge()) {
-        return docWriter.getNextSequenceNumber();
+        // nocommit -- add tests for this transactional behavior
+        synchronized (this) {
+          if (infos.isEmpty() == false) {
+            boolean success = false;
+            try {
+              ensureOpen();
+              // Reserve the docs, just before we update SIS:
+              reserveDocs(totalMaxDoc);
+              success = true;
+            } finally {
+              if (!success) {
+                for (SegmentCommitInfo sipc : infos) {
+                  // Safe: these files must exist
+                  deleteNewFiles(sipc.files());
+                }
+              }
+            }
+            segmentInfos.addAll(infos);
+            checkpoint();
+          }
+          seqNo = docWriter.getNextSequenceNumber();
+        }
+      } else {
+        // We should normally not reach here, as an earlier call should throw an exception.
+        throw new MergePolicy.MergeException("Could not complete merges within configured timeout of [" + mergeTimeoutInSeconds + "] seconds");
       }
+    } catch (VirtualMachineError tragedy) {
+      tragicEvent(tragedy, "addIndexes(CodecReader...)");
+      throw tragedy;
+    }
 
-      synchronized (this) {
-        ensureOpen();
-        assert merges.areEnabled();
-        runningAddIndexesMerges.add(merger);
+    maybeMerge();
+    return seqNo;
+  }
+
+  private class AddIndexesMergeSource implements MergeScheduler.MergeSource {

Review comment:
       Very cool that we are able to use `MergeSource` for this!!  I think this was added to decouple IW and MS so MS could be better tested (by mocking `MergeSource`s).  Wonderful to see that this better decoupling now helps nice changes like this, too.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1068485083


   `./gradlew check` had passed on my local machine. However, some tests have randomization and don't always invoke `addIndexes(CodecReaders...)`, and have failed in the run above.
   
   Is there a way to ensure all code paths in the random tests get executed? I want to run the tests that invoke `addIndexes(CodecReader...)` for some random flag value.


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r827274463



##########
File path: lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
##########
@@ -39,6 +40,11 @@ public MergeSpecification findMerges(
     return null;
   }
 
+  @Override
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    return null;

Review comment:
       There are consumers of `NoMergePolicy` that expect `addIndexes(CodecReaders...)` to work, like `IndexRearranger`. My understanding is that NoMergePolicy is meant to turn off any merges on existing segments. Since this API uses merges to add new readers to the index, I've changed it here to return the default base class impl.
   
   It's a little off because the name is NoMergePolicy, but I think(hope) it aligns with the intent behind that class. I've added some comments to call this out. Let me know if we should instead create a different MergePolicy that only allows fineMerges(readers).. (it would be exactly like NoMergePolicy except for one api).




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822195662



##########
File path: lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
##########
@@ -39,6 +40,11 @@ public MergeSpecification findMerges(
     return null;
   }
 
+  @Override
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    return null;

Review comment:
       In my current impl, addIndexes simply completes without adding anything if the API returns null. I now realize, this is misleading for the user. Callers of addIndexes may not know about the configured merge policy. I should probably throw an exception if I get a null here?
   On similar lines, what should we do if the API call had some CodecReaders, but findMerges returns an empty spec.? Should we error out? (Will add a log for this either way.)
   
   Re: letting the API return null, I see that other variants of `findMerges` in this policy, all return `null`, and all callers of findMerges seem to be doing the null checks. Personally, I prefer disallowing the null return values, but I wanted to keep this in sync with the other variants.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r824243569



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {
+            // nocommit -- ensure all intermediate files are deleted
+            for (MergePolicy.OneMerge merge: spec.merges) {
+              deleteNewFiles(merge.getMergeInfo().files());
+            }
+          }
+        }
+      }
 
-      // TODO: somehow we should fix this merge so it's
-      // abortable so that IW.close(false) is able to stop it
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-      Codec codec = config.getCodec();
-      // We set the min version to null for now, it will be set later by SegmentMerger
-      SegmentInfo info =
-          new SegmentInfo(
-              directoryOrig,
-              Version.LATEST,
-              null,
-              mergedName,
-              -1,
-              false,
-              codec,
-              Collections.emptyMap(),
-              StringHelper.randomId(),
-              Collections.emptyMap(),
-              config.getIndexSort());
-
-      SegmentMerger merger =
-          new SegmentMerger(
-              Arrays.asList(readers), info, infoStream, trackingDir, globalFieldNumberMap, context);
+      if (mergesComplete) {
+        List<SegmentCommitInfo> infos = new ArrayList<>();
+        long totalMaxDoc = 0;
+        for (MergePolicy.OneMerge merge: spec.merges) {
+          totalMaxDoc += merge.totalMaxDoc;
+          if (merge.getMergeInfo() != null) {

Review comment:
       It will be null if the merge never started, for example, say the readers were empty. I'll add a comment.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1068815107


   1. Found a better way to handle addIndexes failures due to background merges -
   OneMerge objects store the error they run into, which can be pulled via `OneMerge.getException()`. We `rethrow` these errors if they turn out to be the cause for addIndexes failure. This fixes tests which expect certain exceptions, like random IO exceptions thrown by MockDirectoryWrapper in `BasePointsFormatTestCase#testWithExceptions`.
   
   2. Added a test for validate we API and destination writer state when we hit `IndexWriter.actualMaxDocs` limit. 
   3. Fixed breaking tests from the last run.


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822175645



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +

Review comment:
       Changed to AlreadyClosedException.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822185980



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));
+      boolean mergesComplete = false;
+      if (spec != null && spec.merges.size() > 0) {
+        try {
+          spec.merges.forEach(addIndexesMergeSource::registerMerge);
+          mergeScheduler.merge(addIndexesMergeSource, MergeTrigger.ADD_INDEXES);
+          mergesComplete = spec.await();
+        } finally {
+          if (!mergesComplete) {
+            // nocommit -- ensure all intermediate files are deleted
+            for (MergePolicy.OneMerge merge: spec.merges) {
+              deleteNewFiles(merge.getMergeInfo().files());
+            }
+          }
+        }
+      }
 
-      // TODO: somehow we should fix this merge so it's
-      // abortable so that IW.close(false) is able to stop it
-      TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-      Codec codec = config.getCodec();
-      // We set the min version to null for now, it will be set later by SegmentMerger
-      SegmentInfo info =
-          new SegmentInfo(
-              directoryOrig,
-              Version.LATEST,
-              null,
-              mergedName,
-              -1,
-              false,
-              codec,
-              Collections.emptyMap(),
-              StringHelper.randomId(),
-              Collections.emptyMap(),
-              config.getIndexSort());
-
-      SegmentMerger merger =
-          new SegmentMerger(
-              Arrays.asList(readers), info, infoStream, trackingDir, globalFieldNumberMap, context);
+      if (mergesComplete) {
+        List<SegmentCommitInfo> infos = new ArrayList<>();
+        long totalMaxDoc = 0;
+        for (MergePolicy.OneMerge merge: spec.merges) {
+          totalMaxDoc += merge.totalMaxDoc;
+          if (merge.getMergeInfo() != null) {
+            infos.add(merge.getMergeInfo());
+          }
+        }
 
-      if (!merger.shouldMerge()) {
-        return docWriter.getNextSequenceNumber();
+        // nocommit -- add tests for this transactional behavior
+        synchronized (this) {
+          if (infos.isEmpty() == false) {
+            boolean success = false;
+            try {
+              ensureOpen();
+              // Reserve the docs, just before we update SIS:
+              reserveDocs(totalMaxDoc);
+              success = true;
+            } finally {
+              if (!success) {
+                for (SegmentCommitInfo sipc : infos) {
+                  // Safe: these files must exist
+                  deleteNewFiles(sipc.files());
+                }
+              }
+            }
+            segmentInfos.addAll(infos);
+            checkpoint();
+          }
+          seqNo = docWriter.getNextSequenceNumber();
+        }
+      } else {
+        // We should normally not reach here, as an earlier call should throw an exception.
+        throw new MergePolicy.MergeException("Could not complete merges within configured timeout of [" + mergeTimeoutInSeconds + "] seconds");
       }
+    } catch (VirtualMachineError tragedy) {
+      tragicEvent(tragedy, "addIndexes(CodecReader...)");
+      throw tragedy;
+    }
 
-      synchronized (this) {
-        ensureOpen();
-        assert merges.areEnabled();
-        runningAddIndexesMerges.add(merger);
+    maybeMerge();
+    return seqNo;
+  }
+
+  private class AddIndexesMergeSource implements MergeScheduler.MergeSource {

Review comment:
       +1, `MergeSource` is a neat abstraction. 
   I wanted to move this out of IW, to a different class altogether - try to make IW leaner. But I ran into a bunch of test failures and method access issues that digressed from the main change. Maybe I'll try it again in a separate, follow up 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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822184400



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));

Review comment:
       We allow for cascading to proceed in background by calling a `maybeMerge()` at the end of `addIndexes()`.
   
   For `addIndexes(CodecReader...)`, we can only add segments to IW after the readers have gone through `SegmentMerger.merge()`. Until then, we don't have any segments to add, (unlike the `addIndexes(Directory...)` variant, where we already have segments provided in the input). So we wait for the first round of merges to complete, by blocking the API on the merges defined in the MergeSpec.
   
   Once these merges complete, we add segments to IW's SegmentInfos, and invoke a maybeMerge(), which will cascade any merges on top of these segments in




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] mikemccand commented on a change in pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r830125445



##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -567,6 +605,21 @@ public abstract MergeSpecification findMerges(
       MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
       throws IOException;
 
+  /**
+   * Define {@link OneMerge} operations for a list of codec readers. This call is used to
+   * define merges for input readers in {@link IndexWriter#addIndexes(CodecReader...)}.
+   * Default implementation adds all readers to a single merge. This can be overridden in custom
+   * merge policies.
+   *
+   * @param readers set of readers to merge into the main index
+   */
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    OneMerge merge = new OneMerge(readers, leaf -> new MergeReader(leaf, leaf.getLiveDocs()));

Review comment:
       > Which, I believe is similar to the behavior in addIndexes(Directory...) - all incoming segments are simply added to IW, and any merging happens in later in background.
   
   Oooh I really like that!  +1 for follow-on issue here to make both versions behave more consistently.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1073177695


   Thanks @mikemccand . I figured that I had changed the exception signature for addIndexes() in my last revision. I was throwing a blanket MergePolicyException when addIndexes failed to merge all the readers in; and a lot of these tests were designed to look for specific exceptions that would've earlier gotten thrown from the SegmentMerger code  running in main addIndexes thread.. This was causing the randomly invoked tests to fail.
   
   I changed that, to `rethrow` the exceptions seen by any failed background OneMerges, and that appeased the gradle gods and builds began to pass again. 
   
   I think this is better, because now the API throws the actual error seen by a thread doing the merge, instead of some blanket failure message. In general, I have to say, I'm fascinated by the breadth of test cases around this API, and the code framework present to add more tests.


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1068299715


   ### New Changes:
    1. `addIndexes(CodecReader...)` merges are abortable now. 
       * Earlier, `addIndexes(CodecReader...)` triggered merges directly via SegmentMerger. As a result, the running merges could be tracked in `Set<SegmentMerger> runningAddIndexesMerges`, but any pending merge operations could not be aborted preemptively.
       * With this change, merges are defined via `MergePolicy.OneMerge` objects, and scheduled by the `MergeScheduler`. These are aborted when IndexWriter is rolled back or closed.
       * removed corresponding TODO - `“TODO: somehow we should fix this merge so it’s abortable so that IW.close(false) is able to stop it”`
   2. Updated NoMergePolicy to return the default MergeSpec for `findMerges(List<CodecReader> readers)`, as it is required by consumers of addIndexes that do use this policy to avoid background segment merges, like `IndexRearranger`.
   
   ### Tests Added:
   1. Test addIndexes with a MergePolicy that creates one merge per CodecReader.
   2. Test transactionality when only a subset of triggered merges succeed.
   3. Test interim files are deleted when a partial subset of merges fails, causing the API to fail.
   4. Tests for null and empty MergeSpec returned from MergePolicy.
   5. Test no merges are triggered if empty readers provided.
   6. Test cascading merges get triggered after each `addIndexes(CodecReader...)` call.
   
   ### Existing Tests Expanded:
   1. Added a test MergePolicy that creates one merge per CodecReader. This is used with multiple existing concurrency tests, that check for steady state calls, concurrent operations overlapping with addIndexes, IW closed (with and without wait) with a conc. running addIndexes, and IW rollback. 
   2. Tested that pending merges get aborted when IW is rolled back.
   


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r827270775



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));

Review comment:
       I've also added a test to verify this.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822190688



##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -813,12 +866,24 @@ protected final boolean verbose(MergeContext mergeContext) {
   }
 
   static final class MergeReader {
+    final CodecReader codecReader;
     final SegmentReader reader;

Review comment:
       So far, I see `SegmentReader` being used only in `ReadersAndUpdates` [here](https://github.com/apache/lucene/blob/main/lucene%2Fcore%2Fsrc%2Fjava%2Forg%2Fapache%2Flucene%2Findex%2FReadersAndUpdates.java#L786-L793). I am not really sure if `createNewReaderWithLatestLiveDocs()` can also work with a CodecReader.. Will look into this more.




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] mikemccand commented on pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1072547237


   > Is there a way to ensure all code paths in the random tests get executed? I want to run the tests that invoke `addIndexes(CodecReader...)` for some random flag value.
   
   Alas, I don't think we have a way to ensure ALL paths get executed -- that would likely be combinatorically prohibitive?  But sure would be nice to have some control over this.
   
   But we do have `gradle beast`, which will run the same test case over and over with different random seeds and hopefully over time provoke any latent failures that one or two successful runs failed to uncover?


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r830568250



##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -813,12 +866,24 @@ protected final boolean verbose(MergeContext mergeContext) {
   }
 
   static final class MergeReader {
+    final CodecReader codecReader;
     final SegmentReader reader;

Review comment:
       I looked into this more, and I'm not able to get rid of either of them in the current structure. SegmentReader seems to hold important information about segments that is required by consumers in different places, like the regular segment merges, so I can't always just use CodecReader in OneMerge objects.
   
   AddIndexes() can only work with CodecReaders, as there are no segments to begin with (which means I can also not create a segment reader from the provided codec readers here).
   
   Maybe, I can extend `OneMerge` to create a `ReaderOneMerge` class that only works with CodecReaders? Would love to hear your thoughts, or any other better ideas..




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1036989788


   The `addIndexes(CodecReaders[])` API is transactional in nature - either all readers get added or all of them fail. Retaining this behavior with concurrent background merges needs some work.
   
   My plan is to defer adding the newly created merged segments to IndexWriter until all merges finish. MergeScheduler threads will do all the work for merge right upto creating segment files, and return. At addIndexes() API, we'll wait for all merges kicked off by the API to complete, and add them to `IndexWriter.segmentInfos` in a sync block only when all of them succeed. For partial failures, we'll delete the interim files created.
   
   Added code for this change to get some feedback on this approach. Tests are failing right now and I'm working on fixing them and adding new ones. Have put `nocommit` comments meanwhile.


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r822184400



##########
File path: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
##########
@@ -3121,147 +3125,265 @@ private void validateMergeReader(CodecReader leaf) {
    */
   public long addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
-
-    // long so we can detect int overflow:
-    long numDocs = 0;
     long seqNo;
-    try {
-      if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "flush at addIndexes(CodecReader...)");
-      }
-      flush(false, true);
+    long numDocs = 0;
+    final int mergeTimeoutInSeconds = 600;
 
-      String mergedName = newSegmentName();
-      int numSoftDeleted = 0;
-      for (CodecReader leaf : readers) {
-        numDocs += leaf.numDocs();
+    try {
+      // Best effort up front validations
+      for (CodecReader leaf: readers) {
         validateMergeReader(leaf);
-        if (softDeletesEnabled) {
-          Bits liveDocs = leaf.getLiveDocs();
-          numSoftDeleted +=
-              PendingSoftDeletes.countSoftDeletes(
-                  DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(
-                      config.getSoftDeletesField(), leaf),
-                  liveDocs);
+        for (FieldInfo fi: leaf.getFieldInfos()) {
+          globalFieldNumberMap.verifyFieldInfo(fi);
         }
+        numDocs += leaf.numDocs();
       }
-
-      // Best-effort up front check:
       testReserveDocs(numDocs);
 
-      final IOContext context =
-          new IOContext(
-              new MergeInfo(Math.toIntExact(numDocs), -1, false, UNBOUNDED_MAX_MERGE_SEGMENTS));
+      synchronized (this) {
+        ensureOpen();
+        if (merges.areEnabled() == false) {
+          throw new UnsupportedOperationException("Merges are disabled on current writer. " +
+            "Cannot execute addIndexes(CodecReaders...) API");
+        }
+      }
+
+      MergePolicy mergePolicy = config.getMergePolicy();
+      MergePolicy.MergeSpecification spec = mergePolicy.findMerges(Arrays.asList(readers));

Review comment:
       We allow for cascading to proceed in background by calling a `maybeMerge()` at the end of `addIndexes()`.
   
   For `addIndexes(CodecReader...)`, we can only add segments to IW after the readers have gone through `SegmentMerger.merge()`. Until then, we don't have any segments to add, (unlike the `addIndexes(Directory...)` variant, where we already have segments provided in the input). So we wait for the first round of merges to complete, by blocking the API on the merges defined in the MergeSpec.
   
   Once these merges complete, we add segments to IW's SegmentInfos, and invoke a maybeMerge(), which will cascade any merges on top of these segments in background




-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1062441177


   Thanks for reviewing this @mikemccand ! I've addressed the ones that had minor changes, and am working on the larger ones.
   
   > What remains to bring this out of WIP?
   
   There are 2 existing tests that are breaking:
   ```java
   - org.apache.lucene.index.TestAddIndexes.testAddIndicesWithSoftDeletes (:lucene:core)
   - org.apache.lucene.index.TestIndexSorting.testAddIndexesWithDeletions (:lucene:core)
   ```
   
   I've been looking at `testAddIndicesWithSoftDeletes`. IW does a `rollbackInternal()` on shutDown, where it is failing on this assert - `assert pendingNumDocs.get() == segmentInfos.totalMaxDoc()`. I'm guessing my code is reserving more docs than it should.. is that the right direction? 
   Here's a longer error stack trace - 
   ```java
   java.lang.AssertionError: pendingNumDocs 7 != 5 totalMaxDoc
      at __randomizedtesting.SeedInfo.seed([63C0554849BFDEC5:4F94AA0CFCBAFADD]:0)
      at org.apache.lucene.index.IndexWriter.rollbackInternal(IndexWriter.java:2428)
      at org.apache.lucene.index.IndexWriter.shutdown(IndexWriter.java:1334)
      at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1362)
      at org.apache.lucene.util.IOUtils.close(IOUtils.java:85)
      at org.apache.lucene.util.IOUtils.close(IOUtils.java:72)
      at org.apache.lucene.index.TestAddIndexes.testAddIndicesWithSoftDeletes(TestAddIndexes.java:1521)
   ```
    
   Outside of fixing existing tests, I want to add some new tests, especially around cases where one of the merges fails, causing the whole API to fail (i.e. test that transactionality was retained).
   
   


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on pull request #633: [WIP] LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on pull request #633:
URL: https://github.com/apache/lucene/pull/633#issuecomment-1068300899


   I think this PR has reasonable test coverage and can come out of WIP now. Will continue to iterate on the PR comments and suggestions.. 


-- 
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-unsubscribe@lucene.apache.org

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



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


[GitHub] [lucene] vigyasharma commented on a change in pull request #633: LUCENE-10216: Use MergeScheduler and MergePolicy to run addIndexes(CodecReader[]) merges.

Posted by GitBox <gi...@apache.org>.
vigyasharma commented on a change in pull request #633:
URL: https://github.com/apache/lucene/pull/633#discussion_r830568703



##########
File path: lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
##########
@@ -567,6 +605,21 @@ public abstract MergeSpecification findMerges(
       MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext)
       throws IOException;
 
+  /**
+   * Define {@link OneMerge} operations for a list of codec readers. This call is used to
+   * define merges for input readers in {@link IndexWriter#addIndexes(CodecReader...)}.
+   * Default implementation adds all readers to a single merge. This can be overridden in custom
+   * merge policies.
+   *
+   * @param readers set of readers to merge into the main index
+   */
+  public MergeSpecification findMerges(List<CodecReader> readers) throws IOException {
+    OneMerge merge = new OneMerge(readers, leaf -> new MergeReader(leaf, leaf.getLiveDocs()));

Review comment:
       Create https://issues.apache.org/jira/browse/LUCENE-10476 for the follow up task.




-- 
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-unsubscribe@lucene.apache.org

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



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