You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2020/06/18 22:46:40 UTC

[lucene-solr] branch branch_8x updated: LUCENE-8962: add ability to selectively merge on commit (#1552)

This is an automated email from the ASF dual-hosted git repository.

sokolov pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 28f2e1c  LUCENE-8962: add ability to selectively merge on commit (#1552)
28f2e1c is described below

commit 28f2e1cd7f39e482b100d55ec94c46fc292f7b67
Author: Michael Sokolov <so...@falutin.net>
AuthorDate: Thu Jun 18 16:56:29 2020 -0400

    LUCENE-8962: add ability to selectively merge on commit (#1552)
    
    Co-authored-by: Michael Froh <ms...@apache.org>
    Co-authored-by: Simon Willnauer <si...@apache.org>
---
 lucene/CHANGES.txt                                 |   4 +-
 .../org/apache/lucene/index/FilterMergePolicy.java |   5 +
 .../java/org/apache/lucene/index/IndexWriter.java  |  66 +++++++-
 .../org/apache/lucene/index/IndexWriterConfig.java |  25 ++++
 .../apache/lucene/index/LiveIndexWriterConfig.java |  13 ++
 .../java/org/apache/lucene/index/MergePolicy.java  |  26 +++-
 .../java/org/apache/lucene/index/MergeTrigger.java |   7 +-
 .../org/apache/lucene/index/NoMergePolicy.java     |   3 +
 .../lucene/index/OneMergeWrappingMergePolicy.java  |   5 +
 .../org/apache/lucene/index/TestIndexWriter.java   | 166 +++++++++++++--------
 .../lucene/index/TestIndexWriterMergePolicy.java   |  71 ++++++++-
 .../apache/lucene/index/MockRandomMergePolicy.java |  32 ++++
 .../org/apache/lucene/util/LuceneTestCase.java     |   1 +
 13 files changed, 350 insertions(+), 74 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6dfe6a2..91c81ae 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -268,7 +268,9 @@ Improvements
 
 * LUCENE-9253: KoreanTokenizer now supports custom dictionaries(system, unknown). (Namgyu Kim)
 
-* LUCENE-8962: Add ability to selectively merge on commit (Michael Froh)
+* LUCENE-8962: Add IndexWriter merge-on-commit feature to selectively merge small segments on commit,
+  subject to a configurable timeout, to improve search performance by reducing the number of small
+  segments for searching (Michael Froh, Mike Mccandless, Simon Willnauer)
 
 * LUCENE-9171: QueryBuilder can now use BoostAttributes on input token streams to selectively
   boost particular terms or synonyms in parsed queries. (Alessandro Benedetti, Alan Woodward)
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
index eb634b4..b4e33f8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
@@ -58,6 +58,11 @@ public class FilterMergePolicy extends MergePolicy {
   }
 
   @Override
+  public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+    return in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext);
+  }
+
+  @Override
   public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext)
       throws IOException {
     return in.useCompoundFile(infos, mergedInfo, mergeContext);
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index f52426f..c6aab90 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -33,6 +33,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -2183,7 +2184,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
         }
       }
     } else {
-      spec = mergePolicy.findMerges(trigger, segmentInfos, this);
+      switch (trigger) {
+        case COMMIT:
+          spec = mergePolicy.findFullFlushMerges(trigger, segmentInfos, this);
+          break;
+        default:
+          spec = mergePolicy.findMerges(trigger, segmentInfos, this);
+      }
     }
     if (spec != null) {
       final int numMerges = spec.merges.size();
@@ -3189,7 +3196,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       SegmentInfos toCommit = null;
       boolean anyChanges = false;
       long seqNo;
-
+      MergePolicy.MergeSpecification onCommitMerges = null;
+      AtomicBoolean includeInCommit = new AtomicBoolean(true);
+      final long maxCommitMergeWaitSeconds = config.getMaxCommitMergeWaitSeconds();
       // This is copied from doFlush, except it's modified to
       // clone & incRef the flushed SegmentInfos inside the
       // sync block:
@@ -3243,6 +3252,45 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
               // sneak into the commit point:
               toCommit = segmentInfos.clone();
 
+              if (anyChanges && maxCommitMergeWaitSeconds > 0) {
+                SegmentInfos committingSegmentInfos = toCommit;
+                onCommitMerges = updatePendingMerges(new OneMergeWrappingMergePolicy(config.getMergePolicy(), toWrap ->
+                    new MergePolicy.OneMerge(toWrap.segments) {
+                      @Override
+                      public void mergeFinished(boolean committed) throws IOException {
+                        assert Thread.holdsLock(IndexWriter.this);
+                        if (committed && includeInCommit.get()) {
+                          deleter.incRef(info.files());
+                          Set<String> mergedSegmentNames = new HashSet<>();
+                          for (SegmentCommitInfo sci : segments) {
+                            mergedSegmentNames.add(sci.info.name);
+                          }
+                          List<SegmentCommitInfo> toCommitMergedAwaySegments = new ArrayList<>();
+                          for (SegmentCommitInfo sci : committingSegmentInfos) {
+                            if (mergedSegmentNames.contains(sci.info.name)) {
+                              toCommitMergedAwaySegments.add(sci);
+                              deleter.decRef(sci.files());
+                            }
+                          }
+                          // Construct a OneMerge that applies to toCommit
+                          MergePolicy.OneMerge applicableMerge = new MergePolicy.OneMerge(toCommitMergedAwaySegments);
+                          applicableMerge.info = info.clone();
+                          long segmentCounter = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+                          committingSegmentInfos.counter = Math.max(committingSegmentInfos.counter, segmentCounter + 1);
+                          committingSegmentInfos.applyMergeChanges(applicableMerge, false);
+                        }
+                        toWrap.mergeFinished(committed);
+                        super.mergeFinished(committed);
+                      }
+
+                      @Override
+                      public CodecReader wrapForMerge(CodecReader reader) throws IOException {
+                        return toWrap.wrapForMerge(reader);
+                      }
+                    }
+                ), MergeTrigger.COMMIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
+              }
+
               pendingCommitChangeCount = changeCount.get();
 
               // This protects the segmentInfos we are now going
@@ -3250,8 +3298,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
               // we are trying to sync all referenced files, a
               // merge completes which would otherwise have
               // removed the files we are now syncing.    
-              filesToCommit = toCommit.files(false); 
-              deleter.incRef(filesToCommit);
+              deleter.incRef(toCommit.files(false));
             }
             success = true;
           } finally {
@@ -3272,7 +3319,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       } finally {
         maybeCloseOnTragicEvent();
       }
-     
+
+      if (onCommitMerges != null) {
+        mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+        onCommitMerges.await(maxCommitMergeWaitSeconds, TimeUnit.SECONDS);
+        synchronized (this) {
+          // we need to call this under lock since mergeFinished above is also called under the IW lock
+          includeInCommit.set(false);
+        }
+      }
+      filesToCommit = toCommit.files(false);
       try {
         if (anyChanges) {
           maybeMerge.set(true);
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
index 26e7e3d..1237932 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -109,6 +110,9 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
   
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+  /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+  public static final long DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 0;
   
   // indicates whether this config instance is already attached to a writer.
   // not final so that it can be cloned properly.
@@ -460,6 +464,27 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
   }
 
   /**
+   * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and will still run to completion independent of the commit
+   * like normal segment merges. The default is <code>{@value IndexWriterConfig#DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS}</code>.
+   *
+   * Note: This settings has no effect unless {@link MergePolicy#findFullFlushMerges(MergeTrigger, SegmentInfos, MergePolicy.MergeContext)}
+   * has an implementation that actually returns merges which by default doesn't return any merges.
+   */
+  public IndexWriterConfig setMaxCommitMergeWaitSeconds(long maxCommitMergeWaitSeconds) {
+    this.maxCommitMergeWaitSeconds = maxCommitMergeWaitSeconds;
+    return this;
+  }
+
+  /** We only allow sorting on these types */
+  private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
+                                                                                     SortField.Type.LONG,
+                                                                                     SortField.Type.INT,
+                                                                                     SortField.Type.DOUBLE,
+                                                                                     SortField.Type.FLOAT);
+
+  /**
    * Set the {@link Sort} order to use for all (flushed and merged) segments.
    */
   public IndexWriterConfig setIndexSort(Sort sort) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
index 1f48acc..9b1d56c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
@@ -109,6 +109,8 @@ public class LiveIndexWriterConfig {
   /** soft deletes field */
   protected String softDeletesField = null;
 
+  /** Amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...) */
+  protected volatile long maxCommitMergeWaitSeconds;
 
   // used by IndexWriterConfig
   LiveIndexWriterConfig(Analyzer analyzer) {
@@ -132,6 +134,7 @@ public class LiveIndexWriterConfig {
     flushPolicy = new FlushByRamOrCountsPolicy();
     readerPooling = IndexWriterConfig.DEFAULT_READER_POOLING;
     perThreadHardLimitMB = IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB;
+    maxCommitMergeWaitSeconds = IndexWriterConfig.DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS;
   }
   
   /** Returns the default analyzer to use for indexing documents. */
@@ -461,6 +464,15 @@ public class LiveIndexWriterConfig {
     return softDeletesField;
   }
 
+  /**
+   * Expert: return the amount of time to wait for merges returned by by MergePolicy.findFullFlushMerges(...).
+   * If this time is reached, we proceed with the commit based on segments merged up to that point.
+   * The merges are not cancelled, and may still run to completion independent of the commit.
+   */
+  public long getMaxCommitMergeWaitSeconds() {
+    return maxCommitMergeWaitSeconds;
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
@@ -484,6 +496,7 @@ public class LiveIndexWriterConfig {
     sb.append("indexSort=").append(getIndexSort()).append("\n");
     sb.append("checkPendingFlushOnUpdate=").append(isCheckPendingFlushOnUpdate()).append("\n");
     sb.append("softDeletesField=").append(getSoftDeletesField()).append("\n");
+    sb.append("maxCommitMergeWaitSeconds=").append(getMaxCommitMergeWaitSeconds()).append("\n");
     return sb.toString();
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
index 560bd8e..d8e7a06 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -555,7 +555,7 @@ public abstract class MergePolicy {
  *          an original segment present in the
  *          to-be-merged index; else, it was a segment
  *          produced by a cascaded merge.
-   * @param mergeContext the IndexWriter to find the merges on
+   * @param mergeContext the MergeContext to find the merges on
    */
   public abstract MergeSpecification findForcedMerges(
       SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, MergeContext mergeContext)
@@ -566,12 +566,34 @@ public abstract class MergePolicy {
    * deletes from the index.
    *  @param segmentInfos
    *          the total set of segments in the index
-   * @param mergeContext the IndexWriter to find the merges on
+   * @param mergeContext the MergeContext to find the merges on
    */
   public abstract MergeSpecification findForcedDeletesMerges(
       SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException;
 
   /**
+   * Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit.
+   *
+   * Any merges returned here will make {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} block until
+   * the merges complete or until {@link IndexWriterConfig#getMaxCommitMergeWaitSeconds()} have elapsed. This may be
+   * used to merge small segments that have just been flushed as part of the commit, reducing the number of segments in
+   * the commit. If a merge does not complete in the allotted time, it will continue to execute, but will not be reflected
+   * in the commit.
+   *
+   * If a {@link OneMerge} in the returned {@link MergeSpecification} includes a segment already included in a registered
+   * merge, then {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} will throw a {@link IllegalStateException}.
+   * Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge.
+   *
+   * @param mergeTrigger the event that triggered the merge (COMMIT or FULL_FLUSH).
+   * @param segmentInfos the total set of segments in the index (while preparing the commit)
+   * @param mergeContext the MergeContext to find the merges on, which should be used to determine which segments are
+ *                     already in a registered merge (see {@link MergeContext#getMergingSegments()}).
+   */
+  public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+    return null;
+  }
+
+  /**
    * Returns true if a new segment (regardless of its origin) should use the
    * compound file format. The default implementation returns <code>true</code>
    * iff the size of the given mergedInfo is less or equal to
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java b/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
index d165a27..01a6b15 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
@@ -47,5 +47,10 @@ public enum MergeTrigger {
   /**
    * Merge was triggered by a closing IndexWriter.
    */
-  CLOSING
+  CLOSING,
+
+  /**
+   * Merge was triggered on commit.
+   */
+  COMMIT,
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
index 1480ce4..b209e8ae 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
@@ -46,6 +46,9 @@ public final class NoMergePolicy extends MergePolicy {
   public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
 
   @Override
+  public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
+
+  @Override
   public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment, MergeContext mergeContext) {
     return newSegment.info.getUseCompoundFile();
   }
diff --git a/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
index d08711e..a5fd66a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
@@ -59,6 +59,11 @@ public class OneMergeWrappingMergePolicy extends FilterMergePolicy {
     return wrapSpec(in.findForcedDeletesMerges(segmentInfos, mergeContext));
   }
 
+  @Override
+  public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+    return wrapSpec(in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext));
+  }
+
   private MergeSpecification wrapSpec(MergeSpecification spec) {
     MergeSpecification wrapped = spec == null ? null : new MergeSpecification();
     if (wrapped != null) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index c4331c7..be41167 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -345,7 +345,7 @@ public class TestIndexWriter extends LuceneTestCase {
   // Make sure it's OK to change RAM buffer size and
   // maxBufferedDocs in a write session
   public void testChangingRAMBuffer() throws IOException {
-    Directory dir = newDirectory();      
+    Directory dir = newDirectory();
     IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
     writer.getConfig().setMaxBufferedDocs(10);
     writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
@@ -608,7 +608,7 @@ public class TestIndexWriter extends LuceneTestCase {
           doc.add(newField("content4", contents, customType));
           type = customType;
         } else
-          type = TextField.TYPE_NOT_STORED; 
+          type = TextField.TYPE_NOT_STORED;
         doc.add(newTextField("content1", contents, Field.Store.NO));
         doc.add(newField("content3", "", customType));
         doc.add(newField("content5", "", type));
@@ -664,13 +664,13 @@ public class TestIndexWriter extends LuceneTestCase {
     writer.close();
     dir.close();
   }
-  
+
   public void testEmptyFieldNameTerms() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
     Document doc = new Document();
     doc.add(newTextField("", "a b c", Field.Store.NO));
-    writer.addDocument(doc);  
+    writer.addDocument(doc);
     writer.close();
     DirectoryReader reader = DirectoryReader.open(dir);
     LeafReader subreader = getOnlyLeafReader(reader);
@@ -682,7 +682,7 @@ public class TestIndexWriter extends LuceneTestCase {
     reader.close();
     dir.close();
   }
-  
+
   public void testEmptyFieldNameWithEmptyTerm() throws IOException {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -691,7 +691,7 @@ public class TestIndexWriter extends LuceneTestCase {
     doc.add(newStringField("", "a", Field.Store.NO));
     doc.add(newStringField("", "b", Field.Store.NO));
     doc.add(newStringField("", "c", Field.Store.NO));
-    writer.addDocument(doc);  
+    writer.addDocument(doc);
     writer.close();
     DirectoryReader reader = DirectoryReader.open(dir);
     LeafReader subreader = getOnlyLeafReader(reader);
@@ -835,7 +835,7 @@ public class TestIndexWriter extends LuceneTestCase {
     customType.setStoreTermVectors(true);
     customType.setStoreTermVectorPositions(true);
     customType.setStoreTermVectorOffsets(true);
-    
+
     doc.add(newField("content", "aaa bbb ccc ddd eee fff ggg hhh iii", customType));
     writer.addDocument(doc);
     writer.addDocument(doc);
@@ -923,7 +923,7 @@ public class TestIndexWriter extends LuceneTestCase {
 
       // open/close slowly sometimes
       dir.setUseSlowOpenClosers(true);
-      
+
       // throttle a little
       dir.setThrottling(MockDirectoryWrapper.Throttling.SOMETIMES);
 
@@ -1149,7 +1149,7 @@ public class TestIndexWriter extends LuceneTestCase {
 
     FieldType customType = new FieldType(StoredField.TYPE);
     customType.setTokenized(true);
-    
+
     Field f = new Field("binary", b, 10, 17, customType);
     // TODO: this is evil, changing the type after creating the field:
     customType.setIndexOptions(IndexOptions.DOCS);
@@ -1158,7 +1158,7 @@ public class TestIndexWriter extends LuceneTestCase {
     f.setTokenStream(doc1field1);
 
     FieldType customType2 = new FieldType(TextField.TYPE_STORED);
-    
+
     Field f2 = newField("string", "value", customType2);
     final MockTokenizer doc1field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
     doc1field2.setReader(new StringReader("doc1field2"));
@@ -1234,7 +1234,7 @@ public class TestIndexWriter extends LuceneTestCase {
   public void testDeleteUnusedFiles() throws Exception {
     assumeFalse("test relies on exact filenames", Codec.getDefault() instanceof SimpleTextCodec);
     assumeWorkingMMapOnWindows();
-    
+
     for(int iter=0;iter<2;iter++) {
       // relies on windows semantics
       Path path = createTempDir();
@@ -1251,7 +1251,7 @@ public class TestIndexWriter extends LuceneTestCase {
       }
 
       MergePolicy mergePolicy = newLogMergePolicy(true);
-      
+
       // This test expects all of its segments to be in CFS
       mergePolicy.setNoCFSRatio(1.0);
       mergePolicy.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
@@ -1339,7 +1339,7 @@ public class TestIndexWriter extends LuceneTestCase {
     customType.setStoreTermVectors(true);
     customType.setStoreTermVectorPositions(true);
     customType.setStoreTermVectorOffsets(true);
-    
+
     doc.add(newField("c", "val", customType));
     writer.addDocument(doc);
     writer.commit();
@@ -1380,7 +1380,7 @@ public class TestIndexWriter extends LuceneTestCase {
     // indexed, flushed (but not committed) and then IW rolls back, then no
     // files are left in the Directory.
     Directory dir = newDirectory();
-    
+
     String[] origFiles = dir.listAll();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
                                                 .setMaxBufferedDocs(2)
@@ -1410,8 +1410,8 @@ public class TestIndexWriter extends LuceneTestCase {
     // Adding just one document does not call flush yet.
     int computedExtraFileCount = 0;
     for (String file : dir.listAll()) {
-      if (IndexWriter.WRITE_LOCK_NAME.equals(file) || 
-          file.startsWith(IndexFileNames.SEGMENTS) || 
+      if (IndexWriter.WRITE_LOCK_NAME.equals(file) ||
+          file.startsWith(IndexFileNames.SEGMENTS) ||
           IndexFileNames.CODEC_FILE_PATTERN.matcher(file).matches()) {
         if (file.lastIndexOf('.') < 0
             // don't count stored fields and term vectors in, or any temporary files they might
@@ -1459,7 +1459,7 @@ public class TestIndexWriter extends LuceneTestCase {
     FieldType customType3 = new FieldType(TextField.TYPE_STORED);
     customType3.setTokenized(false);
     customType3.setOmitNorms(true);
-    
+
     for (int i=0; i<2; i++) {
       Document doc = new Document();
       doc.add(new Field("id", Integer.toString(i)+BIG, customType3));
@@ -1479,7 +1479,7 @@ public class TestIndexWriter extends LuceneTestCase {
       SegmentReader sr = (SegmentReader) ctx.reader();
       assertFalse(sr.getFieldInfos().hasVectors());
     }
-    
+
     r0.close();
     dir.close();
   }
@@ -1502,7 +1502,7 @@ public class TestIndexWriter extends LuceneTestCase {
 
     @Override
     public final boolean incrementToken() {
-      clearAttributes();      
+      clearAttributes();
       if (upto < tokens.length) {
         termAtt.setEmpty();
         termAtt.append(tokens[upto]);
@@ -1725,7 +1725,7 @@ public class TestIndexWriter extends LuceneTestCase {
     r.close();
     dir.close();
   }
-  
+
   public void testDontInvokeAnalyzerForUnAnalyzedFields() throws Exception {
     Analyzer analyzer = new Analyzer() {
       @Override
@@ -1760,13 +1760,13 @@ public class TestIndexWriter extends LuceneTestCase {
     w.close();
     dir.close();
   }
-  
+
   //LUCENE-1468 -- make sure opening an IndexWriter with
   // create=true does not remove non-index files
-  
+
   public void testOtherFiles() throws Throwable {
     Directory dir = newDirectory();
-    IndexWriter iw = new IndexWriter(dir, 
+    IndexWriter iw = new IndexWriter(dir,
         newIndexWriterConfig(new MockAnalyzer(random())));
     iw.addDocument(new Document());
     iw.close();
@@ -1775,15 +1775,15 @@ public class TestIndexWriter extends LuceneTestCase {
       IndexOutput out = dir.createOutput("myrandomfile", newIOContext(random()));
       out.writeByte((byte) 42);
       out.close();
-      
+
       new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))).close();
-      
+
       assertTrue(slowFileExists(dir, "myrandomfile"));
     } finally {
       dir.close();
     }
   }
-  
+
   // LUCENE-3849
   public void testStopwordsPosIncHole() throws Exception {
     Directory dir = newDirectory();
@@ -1812,7 +1812,7 @@ public class TestIndexWriter extends LuceneTestCase {
     ir.close();
     dir.close();
   }
-  
+
   // LUCENE-3849
   public void testStopwordsPosIncHole2() throws Exception {
     // use two stopfilters for testing here
@@ -1844,23 +1844,23 @@ public class TestIndexWriter extends LuceneTestCase {
     ir.close();
     dir.close();
   }
-  
+
   // LUCENE-4575
   public void testCommitWithUserDataOnly() throws Exception {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(null));
     writer.commit(); // first commit to complete IW create transaction.
-    
+
     // this should store the commit data, even though no other changes were made
     writer.setLiveCommitData(new HashMap<String,String>() {{
       put("key", "value");
     }}.entrySet());
     writer.commit();
-    
+
     DirectoryReader r = DirectoryReader.open(dir);
     assertEquals("value", r.getIndexCommit().getUserData().get("key"));
     r.close();
-    
+
     // now check setCommitData and prepareCommit/commit sequence
     writer.setLiveCommitData(new HashMap<String,String>() {{
       put("key", "value1");
@@ -1874,7 +1874,7 @@ public class TestIndexWriter extends LuceneTestCase {
     r = DirectoryReader.open(dir);
     assertEquals("value1", r.getIndexCommit().getUserData().get("key"));
     r.close();
-    
+
     // now should commit the second commitData - there was a bug where 
     // IndexWriter.finishCommit overrode the second commitData
     writer.commit();
@@ -1882,7 +1882,7 @@ public class TestIndexWriter extends LuceneTestCase {
     assertEquals("IndexWriter.finishCommit may have overridden the second commitData",
         "value2", r.getIndexCommit().getUserData().get("key"));
     r.close();
-    
+
     writer.close();
     dir.close();
   }
@@ -1897,7 +1897,7 @@ public class TestIndexWriter extends LuceneTestCase {
     }
     return data;
   }
-  
+
   @Test
   public void testGetCommitData() throws Exception {
     Directory dir = newDirectory();
@@ -1907,16 +1907,16 @@ public class TestIndexWriter extends LuceneTestCase {
     }}.entrySet());
     assertEquals("value", getLiveCommitData(writer).get("key"));
     writer.close();
-    
+
     // validate that it's also visible when opening a new IndexWriter
     writer = new IndexWriter(dir, newIndexWriterConfig(null)
                                     .setOpenMode(OpenMode.APPEND));
     assertEquals("value", getLiveCommitData(writer).get("key"));
     writer.close();
-    
+
     dir.close();
   }
-  
+
   public void testNullAnalyzer() throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig iwConf = newIndexWriterConfig(null);
@@ -1943,7 +1943,7 @@ public class TestIndexWriter extends LuceneTestCase {
     iw.close();
     dir.close();
   }
-  
+
   public void testNullDocument() throws IOException {
     Directory dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
@@ -1968,7 +1968,7 @@ public class TestIndexWriter extends LuceneTestCase {
     iw.close();
     dir.close();
   }
-  
+
   public void testNullDocuments() throws IOException {
     Directory dir = newDirectory();
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
@@ -1993,7 +1993,7 @@ public class TestIndexWriter extends LuceneTestCase {
     iw.close();
     dir.close();
   }
-  
+
   public void testIterableFieldThrowsException() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2001,7 +2001,7 @@ public class TestIndexWriter extends LuceneTestCase {
     int docCount = 0;
     int docId = 0;
     Set<String> liveIds = new HashSet<>();
-    for (int i = 0; i < iters; i++) {      
+    for (int i = 0; i < iters; i++) {
       int numDocs = atLeast(4);
       for (int j = 0; j < numDocs; j++) {
         String id = Integer.toString(docId++);
@@ -2009,7 +2009,7 @@ public class TestIndexWriter extends LuceneTestCase {
         fields.add(new StringField("id", id, Field.Store.YES));
         fields.add(new StringField("foo", TestUtil.randomSimpleString(random()), Field.Store.NO));
         docId++;
-        
+
         boolean success = false;
         try {
           w.addDocument(new RandomFailingIterable<IndexableField>(fields, random()));
@@ -2041,7 +2041,7 @@ public class TestIndexWriter extends LuceneTestCase {
     w.close();
     IOUtils.close(reader, dir);
   }
-  
+
   public void testIterableThrowsException() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2089,7 +2089,7 @@ public class TestIndexWriter extends LuceneTestCase {
     w.close();
     IOUtils.close(reader, dir);
   }
-  
+
   public void testIterableThrowsException2() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2129,7 +2129,7 @@ public class TestIndexWriter extends LuceneTestCase {
       this.list = list;
       this.failOn = random.nextInt(5);
     }
-    
+
     @Override
     public Iterator<T> iterator() {
       final Iterator<? extends T> docIter = list.iterator();
@@ -2255,7 +2255,7 @@ public class TestIndexWriter extends LuceneTestCase {
     writer.close();
     dir.close();
   }
-  
+
   public void testMergeAllDeleted() throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
@@ -2478,12 +2478,12 @@ public class TestIndexWriter extends LuceneTestCase {
     IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
     w.addDocument(new Document());
     w.close();
-    
+
     SegmentInfos sis = SegmentInfos.readLatestCommit(d);
     byte[] id1 = sis.getId();
     assertNotNull(id1);
     assertEquals(StringHelper.ID_LENGTH, id1.length);
-    
+
     byte[] id2 = sis.info(0).info.getId();
     byte[] sciId2 = sis.info(0).getId();
     assertNotNull(id2);
@@ -2515,7 +2515,7 @@ public class TestIndexWriter extends LuceneTestCase {
       ids.add(id);
     }
   }
-  
+
   public void testEmptyNorm() throws Exception {
     Directory d = newDirectory();
     IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2580,7 +2580,7 @@ public class TestIndexWriter extends LuceneTestCase {
     assertEquals(1, r2.getIndexCommit().getGeneration());
     assertEquals("segments_1", r2.getIndexCommit().getSegmentsFileName());
     r2.close();
-    
+
     // make a change and another commit
     w.addDocument(new Document());
     w.commit();
@@ -2867,7 +2867,7 @@ public class TestIndexWriter extends LuceneTestCase {
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter w = new IndexWriter(dir, iwc);
     w.close();
-    
+
     IndexOutput out = dir.createTempOutput("_0", "bkd", IOContext.DEFAULT);
     String tempName = out.getName();
     out.close();
@@ -3152,7 +3152,7 @@ public class TestIndexWriter extends LuceneTestCase {
     expectThrows(IllegalArgumentException.class, () -> {
       writer.softUpdateDocument(null, new Document(), new NumericDocValuesField("soft_delete", 1));
     });
-    
+
     expectThrows(IllegalArgumentException.class, () -> {
       writer.softUpdateDocument(new Term("id", "1"), new Document());
     });
@@ -4187,19 +4187,57 @@ public class TestIndexWriter extends LuceneTestCase {
               }
             };
           })))) {
-          Document doc = new Document();
-          doc.add(new StringField("id", "1", Field.Store.NO));
-          writer.addDocument(doc);
-          writer.flush();
-          writer.addDocument(doc);
-          writer.flush();
-          writer.deleteDocuments(new Term("id", "1"));
-          writer.flush();
-          assertEquals(2, writer.getSegmentCount());
-          assertEquals(0, writer.getDocStats().numDocs);
-          assertEquals(2, writer.getDocStats().maxDoc);
-          writer.forceMerge(1);
+        Document doc = new Document();
+        doc.add(new StringField("id", "1", Field.Store.NO));
+        writer.addDocument(doc);
+        writer.flush();
+        writer.addDocument(doc);
+        writer.flush();
+        writer.deleteDocuments(new Term("id", "1"));
+        writer.flush();
+        assertEquals(2, writer.getSegmentCount());
+        assertEquals(0, writer.getDocStats().numDocs);
+        assertEquals(2, writer.getDocStats().maxDoc);
+        writer.forceMerge(1);
       }
     }
   }
+
+  public void testMergeOnCommitKeepFullyDeletedSegments() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    iwc.setMaxCommitMergeWaitSeconds(30);
+    iwc.mergePolicy = new FilterMergePolicy(newMergePolicy()) {
+      @Override
+      public boolean keepFullyDeletedSegment(IOSupplier<CodecReader> readerIOSupplier) {
+        return true;
+      }
+
+      @Override
+      public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger,
+                                                    SegmentInfos segmentInfos,
+                                                    MergeContext mergeContext) {
+        List<SegmentCommitInfo> fullyDeletedSegments = segmentInfos.asList().stream()
+            .filter(s -> s.info.maxDoc() - s.getDelCount() == 0)
+            .collect(Collectors.toList());
+        if (fullyDeletedSegments.isEmpty()) {
+          return null;
+        }
+        MergeSpecification spec = new MergeSpecification();
+        spec.add(new OneMerge(fullyDeletedSegments));
+        return spec;
+      }
+    };
+    IndexWriter w = new IndexWriter(dir, iwc);
+    Document d = new Document();
+    d.add(new StringField("id", "1", Field.Store.YES));
+    w.addDocument(d);
+    w.commit();
+    w.updateDocument(new Term("id", "1"), d);
+    w.commit();
+    try (DirectoryReader reader = w.getReader()) {
+      assertEquals(1, reader.numDocs());
+    }
+    IOUtils.close(w, dir);
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
index ce591a2..2577f6b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
@@ -18,17 +18,42 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.store.Directory;
 
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestIndexWriterMergePolicy extends LuceneTestCase {
-  
+
+  private static final MergePolicy MERGE_ON_COMMIT_POLICY = new LogDocMergePolicy() {
+    @Override
+    public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) {
+      // Optimize down to a single segment on commit
+      if (mergeTrigger == MergeTrigger.COMMIT && segmentInfos.size() > 1) {
+        List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
+        for (SegmentCommitInfo sci : segmentInfos) {
+          if (mergeContext.getMergingSegments().contains(sci) == false) {
+            nonMergingSegments.add(sci);
+          }
+        }
+        if (nonMergingSegments.size() > 1) {
+          MergeSpecification mergeSpecification = new MergeSpecification();
+          mergeSpecification.add(new OneMerge(nonMergingSegments));
+          return mergeSpecification;
+        }
+      }
+      return null;
+    }
+  };
+
   // Test the normal case
   public void testNormalCase() throws IOException {
     Directory dir = newDirectory();
@@ -278,6 +303,50 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
     assertSetters(new LogDocMergePolicy());
   }
 
+  // Test basic semantics of merge on commit
+  public void testMergeOnCommit() throws IOException {
+    Directory dir = newDirectory();
+
+    IndexWriter firstWriter = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+        .setMergePolicy(NoMergePolicy.INSTANCE));
+    for (int i = 0; i < 5; i++) {
+      TestIndexWriter.addDoc(firstWriter);
+      firstWriter.flush();
+    }
+    DirectoryReader firstReader = DirectoryReader.open(firstWriter);
+    assertEquals(5, firstReader.leaves().size());
+    firstReader.close();
+    firstWriter.close(); // When this writer closes, it does not merge on commit.
+
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
+        .setMergePolicy(MERGE_ON_COMMIT_POLICY).setMaxCommitMergeWaitSeconds(30);
+
+
+    IndexWriter writerWithMergePolicy = new IndexWriter(dir, iwc);
+    writerWithMergePolicy.commit(); // No changes. Commit doesn't trigger a merge.
+
+    DirectoryReader unmergedReader = DirectoryReader.open(writerWithMergePolicy);
+    assertEquals(5, unmergedReader.leaves().size());
+    unmergedReader.close();
+
+    TestIndexWriter.addDoc(writerWithMergePolicy);
+    writerWithMergePolicy.commit(); // Doc added, do merge on commit.
+    assertEquals(1, writerWithMergePolicy.getSegmentCount()); //
+
+    DirectoryReader mergedReader = DirectoryReader.open(writerWithMergePolicy);
+    assertEquals(1, mergedReader.leaves().size());
+    mergedReader.close();
+
+    try (IndexReader reader = writerWithMergePolicy.getReader()) {
+      IndexSearcher searcher = new IndexSearcher(reader);
+      assertEquals(6, reader.numDocs());
+      assertEquals(6, searcher.count(new MatchAllDocsQuery()));
+    }
+
+    writerWithMergePolicy.close();
+    dir.close();
+  }
+
   private void assertSetters(MergePolicy lmp) {
     lmp.setMaxCFSSegmentSizeMB(2.0);
     assertEquals(2.0, lmp.getMaxCFSSegmentSizeMB(), EPSILON);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java b/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
index beb4dad..92ffc73 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
@@ -129,6 +129,38 @@ public class MockRandomMergePolicy extends MergePolicy {
   }
 
   @Override
+  public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+    MergeSpecification mergeSpecification = findMerges(null, segmentInfos, mergeContext);
+    if (mergeSpecification == null) {
+      return null;
+    }
+    // Do not return any merges involving already-merging segments.
+    MergeSpecification filteredMergeSpecification = new MergeSpecification();
+    for (OneMerge oneMerge : mergeSpecification.merges) {
+      boolean filtered = false;
+      List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
+      for (SegmentCommitInfo sci : oneMerge.segments) {
+        if (mergeContext.getMergingSegments().contains(sci) == false) {
+          nonMergingSegments.add(sci);
+        } else {
+          filtered = true;
+        }
+      }
+      if (filtered == true) {
+        if (nonMergingSegments.size() > 0) {
+          filteredMergeSpecification.add(new OneMerge(nonMergingSegments));
+        }
+      } else {
+        filteredMergeSpecification.add(oneMerge);
+      }
+    }
+    if (filteredMergeSpecification.merges.size() > 0) {
+      return filteredMergeSpecification;
+    }
+    return null;
+  }
+
+  @Override
   public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) throws IOException {
     // 80% of the time we create CFS:
     return random.nextInt(5) != 1;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 4a94fe8..44e87b6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1006,6 +1006,7 @@ public abstract class LuceneTestCase extends Assert {
     if (rarely(r)) {
       c.setCheckPendingFlushUpdate(false);
     }
+    c.setMaxCommitMergeWaitSeconds(atLeast(r, 1));
     return c;
   }