You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2017/06/21 17:47:29 UTC

[4/5] lucene-solr:master: LUCENE-7868: use multiple threads to concurrently resolve deletes and DV udpates

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index c929ba2..94ffba7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -96,15 +96,18 @@ class DocumentsWriterPerThread {
     final FieldInfos fieldInfos;
     final FrozenBufferedUpdates segmentUpdates;
     final MutableBits liveDocs;
+    final Sorter.DocMap sortMap;
     final int delCount;
 
-    private FlushedSegment(SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
-                           BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount) {
+    private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
+                           BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount, Sorter.DocMap sortMap)
+      throws IOException {
       this.segmentInfo = segmentInfo;
       this.fieldInfos = fieldInfos;
-      this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(segmentUpdates, true) : null;
+      this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(infoStream, segmentUpdates, segmentInfo) : null;
       this.liveDocs = liveDocs;
       this.delCount = delCount;
+      this.sortMap = sortMap;
     }
   }
 
@@ -142,7 +145,7 @@ class DocumentsWriterPerThread {
   SegmentWriteState flushState;
   // Updates for our still-in-RAM (to be flushed next) segment
   final BufferedUpdates pendingUpdates;
-  private final SegmentInfo segmentInfo;     // Current segment we are working on
+  final SegmentInfo segmentInfo;     // Current segment we are working on
   boolean aborted = false;   // True if we aborted
 
   private final FieldInfos.Builder fieldInfos;
@@ -395,7 +398,7 @@ class DocumentsWriterPerThread {
    * {@link DocumentsWriterDeleteQueue}s global buffer and apply all pending
    * deletes to this DWPT.
    */
-  FrozenBufferedUpdates prepareFlush() {
+  FrozenBufferedUpdates prepareFlush() throws IOException {
     assert numDocsInRAM > 0;
     final FrozenBufferedUpdates globalUpdates = deleteQueue.freezeGlobalBuffer(deleteSlice);
     /* deleteSlice can possibly be null if we have hit non-aborting exceptions during indexing and never succeeded 
@@ -421,14 +424,14 @@ class DocumentsWriterPerThread {
     // Apply delete-by-docID now (delete-byDocID only
     // happens when an exception is hit processing that
     // doc, eg if analyzer has some problem w/ the text):
-    if (pendingUpdates.docIDs.size() > 0) {
+    if (pendingUpdates.deleteDocIDs.size() > 0) {
       flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
-      for(int delDocID : pendingUpdates.docIDs) {
+      for(int delDocID : pendingUpdates.deleteDocIDs) {
         flushState.liveDocs.clear(delDocID);
       }
-      flushState.delCountOnFlush = pendingUpdates.docIDs.size();
-      pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.docIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
-      pendingUpdates.docIDs.clear();
+      flushState.delCountOnFlush = pendingUpdates.deleteDocIDs.size();
+      pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.deleteDocIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
+      pendingUpdates.deleteDocIDs.clear();
     }
 
     if (aborted) {
@@ -446,7 +449,8 @@ class DocumentsWriterPerThread {
     final Sorter.DocMap sortMap;
     try {
       sortMap = consumer.flush(flushState);
-      pendingUpdates.terms.clear();
+      // We clear this here because we already resolved them (private to this segment) when writing postings:
+      pendingUpdates.clearDeleteTerms();
       segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
 
       final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
@@ -463,7 +467,7 @@ class DocumentsWriterPerThread {
       }
 
       final BufferedUpdates segmentDeletes;
-      if (pendingUpdates.queries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
+      if (pendingUpdates.deleteQueries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
         pendingUpdates.clear();
         segmentDeletes = null;
       } else {
@@ -480,13 +484,14 @@ class DocumentsWriterPerThread {
 
       assert segmentInfo != null;
 
-      FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
-                                             segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
+      FlushedSegment fs = new FlushedSegment(infoStream, segmentInfoPerCommit, flushState.fieldInfos,
+                                             segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush,
+                                             sortMap);
       sealFlushedSegment(fs, sortMap);
       if (infoStream.isEnabled("DWPT")) {
         infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0)/1000000.0) + " msec");
       }
-
+      
       return fs;
     } catch (Throwable th) {
       abort();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
index cc72342..de5b3fe 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
@@ -16,11 +16,12 @@
  */
 package org.apache.lucene.index;
 
-import org.apache.lucene.util.ThreadInterruptedException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.lucene.util.ThreadInterruptedException;
+
 /**
  * {@link DocumentsWriterPerThreadPool} controls {@link ThreadState} instances
  * and their thread assignments during indexing. Each {@link ThreadState} holds
@@ -163,6 +164,9 @@ final class DocumentsWriterPerThreadPool {
     // don't recycle DWPT by default
   }
 
+  // TODO: maybe we should try to do load leveling here: we want roughly even numbers
+  // of items (docs, deletes, DV updates) to most take advantage of concurrency while flushing
+
   /** This method is used by DocumentsWriter/FlushControl to obtain a ThreadState to do an indexing operation (add/updateDocument). */
   ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter) {
     ThreadState threadState = null;
@@ -208,9 +212,8 @@ final class DocumentsWriterPerThreadPool {
     state.unlock();
     synchronized (this) {
       freeList.add(state);
-      // In case any thread is waiting, wake one of them up since we just released a thread state; notify() should be sufficient but we do
-      // notifyAll defensively:
-      notifyAll();
+      // In case any thread is waiting, wake one of them up since we just released a thread state:
+      notify();
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
index a85c98b..ffd9501 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
@@ -29,8 +29,7 @@ import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
  * <li>
  * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
  * - applies pending delete operations based on the global number of buffered
- * delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is
- * enabled</li>
+ * delete terms if the consumed memory is greater than {@link IndexWriterConfig#getRAMBufferSizeMB()}</li>.
  * <li>
  * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
  * - flushes either on the number of documents per
@@ -60,21 +59,12 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
 
   @Override
   public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
-    if (flushOnDeleteTerms()) {
-      // Flush this state by num del terms
-      final int maxBufferedDeleteTerms = indexWriterConfig
-          .getMaxBufferedDeleteTerms();
-      if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) {
-        control.setApplyAllDeletes();
+    if ((flushOnRAM() && control.getDeleteBytesUsed() > 1024*1024*indexWriterConfig.getRAMBufferSizeMB())) {
+      control.setApplyAllDeletes();
+      if (infoStream.isEnabled("FP")) {
+        infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
       }
     }
-    if ((flushOnRAM() &&
-        control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
-      control.setApplyAllDeletes();
-     if (infoStream.isEnabled("FP")) {
-       infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
-     }
-   }
   }
 
   @Override
@@ -116,15 +106,6 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
 
   /**
    * Returns <code>true</code> if this {@link FlushPolicy} flushes on
-   * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise
-   * <code>false</code>.
-   */
-  protected boolean flushOnDeleteTerms() {
-    return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
-  }
-
-  /**
-   * Returns <code>true</code> if this {@link FlushPolicy} flushes on
    * {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise
    * <code>false</code>.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
index e70959f..cad07b4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
@@ -33,9 +33,6 @@ import org.apache.lucene.util.InfoStream;
  * <li>Number of RAM resident documents - configured via
  * {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
  * </ul>
- * The policy also applies pending delete operations (by term and/or query),
- * given the threshold set in
- * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}.
  * <p>
  * {@link IndexWriter} consults the provided {@link FlushPolicy} to control the
  * flushing process. The policy is informed for each added or updated document

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index 1ca2830..d953f8d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -34,11 +34,10 @@ final class FreqProxTermsWriter extends TermsHash {
   }
 
   private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
-
     // Process any pending Term deletes for this newly
     // flushed segment:
-    if (state.segUpdates != null && state.segUpdates.terms.size() > 0) {
-      Map<Term,Integer> segDeletes = state.segUpdates.terms;
+    if (state.segUpdates != null && state.segUpdates.deleteTerms.size() > 0) {
+      Map<Term,Integer> segDeletes = state.segUpdates.deleteTerms;
       List<Term> deleteTerms = new ArrayList<>(segDeletes.keySet());
       Collections.sort(deleteTerms);
       String lastField = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
index 4f482ad..d5d1de0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
@@ -16,167 +16,846 @@
  */
 package org.apache.lucene.index;
 
-import java.util.ArrayList;
-import java.util.Iterator;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 
-import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit;
 import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
 import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
-import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Holds buffered deletes and updates by term or query, once pushed. Pushed
  * deletes/updates are write-once, so we shift to more memory efficient data
- * structure to hold them. We don't hold docIDs because these are applied on
+ * structure to hold them.  We don't hold docIDs because these are applied on
  * flush.
  */
 class FrozenBufferedUpdates {
 
+  /* NOTE: we now apply this frozen packet immediately on creation, yet this process is heavy, and runs
+   * in multiple threads, and this compression is sizable (~8.3% of the original size), so it's important
+   * we run this before applying the deletes/updates. */
+
   /* Query we often undercount (say 24 bytes), plus int. */
   final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + Integer.BYTES + 24;
   
   // Terms, in sorted order:
-  final PrefixCodedTerms terms;
+  final PrefixCodedTerms deleteTerms;
 
   // Parallel array of deleted query, and the docIDUpto for each
-  final Query[] queries;
-  final int[] queryLimits;
+  final Query[] deleteQueries;
+  final int[] deleteQueryLimits;
   
   // numeric DV update term and their updates
-  final NumericDocValuesUpdate[] numericDVUpdates;
+  final byte[] numericDVUpdates;
   
   // binary DV update term and their updates
-  final BinaryDocValuesUpdate[] binaryDVUpdates;
+  final byte[] binaryDVUpdates;
+
+  private int numericDVUpdateCount;
+  private int binaryDVUpdateCount;
+
+  /** Counts down once all deletes/updates have been applied */
+  public final CountDownLatch applied = new CountDownLatch(1);
+
+  /** How many total documents were deleted/updated. */
+  public long totalDelCount;
   
   final int bytesUsed;
   final int numTermDeletes;
-  private long gen = -1; // assigned by BufferedUpdatesStream once pushed
-  final boolean isSegmentPrivate;  // set to true iff this frozen packet represents 
-                                   // a segment private deletes. in that case is should
-                                   // only have Queries 
 
+  private long delGen = -1; // assigned by BufferedUpdatesStream once pushed
+
+  final SegmentCommitInfo privateSegment;  // non-null iff this frozen packet represents 
+                                   // a segment private deletes. in that case is should
+                                   // only have Queries and doc values updates
+  private final InfoStream infoStream;
 
-  public FrozenBufferedUpdates(BufferedUpdates deletes, boolean isSegmentPrivate) {
-    this.isSegmentPrivate = isSegmentPrivate;
-    assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries"; 
-    Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]);
+  public FrozenBufferedUpdates(InfoStream infoStream, BufferedUpdates updates, SegmentCommitInfo privateSegment) throws IOException {
+    this.infoStream = infoStream;
+    this.privateSegment = privateSegment;
+    assert updates.deleteDocIDs.isEmpty();
+    assert privateSegment == null || updates.deleteTerms.isEmpty() : "segment private packet should only have del queries"; 
+    Term termsArray[] = updates.deleteTerms.keySet().toArray(new Term[updates.deleteTerms.size()]);
     ArrayUtil.timSort(termsArray);
     PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
     for (Term term : termsArray) {
       builder.add(term);
     }
-    terms = builder.finish();
+    deleteTerms = builder.finish();
     
-    queries = new Query[deletes.queries.size()];
-    queryLimits = new int[deletes.queries.size()];
+    deleteQueries = new Query[updates.deleteQueries.size()];
+    deleteQueryLimits = new int[updates.deleteQueries.size()];
     int upto = 0;
-    for(Map.Entry<Query,Integer> ent : deletes.queries.entrySet()) {
-      queries[upto] = ent.getKey();
-      queryLimits[upto] = ent.getValue();
+    for(Map.Entry<Query,Integer> ent : updates.deleteQueries.entrySet()) {
+      deleteQueries[upto] = ent.getKey();
+      deleteQueryLimits[upto] = ent.getValue();
       upto++;
     }
 
     // TODO if a Term affects multiple fields, we could keep the updates key'd by Term
     // so that it maps to all fields it affects, sorted by their docUpto, and traverse
     // that Term only once, applying the update to all fields that still need to be
-    // updated. 
-    List<NumericDocValuesUpdate> allNumericUpdates = new ArrayList<>();
-    int numericUpdatesSize = 0;
-    for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : deletes.numericUpdates.values()) {
-      for (NumericDocValuesUpdate update : numericUpdates.values()) {
-        allNumericUpdates.add(update);
-        numericUpdatesSize += update.sizeInBytes();
-      }
-    }
-    numericDVUpdates = allNumericUpdates.toArray(new NumericDocValuesUpdate[allNumericUpdates.size()]);
+    // updated.
+    numericDVUpdates = freezeNumericDVUpdates(updates.numericUpdates);
     
     // TODO if a Term affects multiple fields, we could keep the updates key'd by Term
     // so that it maps to all fields it affects, sorted by their docUpto, and traverse
     // that Term only once, applying the update to all fields that still need to be
     // updated. 
-    List<BinaryDocValuesUpdate> allBinaryUpdates = new ArrayList<>();
-    int binaryUpdatesSize = 0;
-    for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : deletes.binaryUpdates.values()) {
+    binaryDVUpdates = freezeBinaryDVUpdates(updates.binaryUpdates);
+    
+    bytesUsed = (int) (deleteTerms.ramBytesUsed() + deleteQueries.length * BYTES_PER_DEL_QUERY 
+                       + numericDVUpdates.length + binaryDVUpdates.length);
+    
+    numTermDeletes = updates.numTermDeletes.get();
+    if (infoStream != null && infoStream.isEnabled("BD")) {
+      infoStream.message("BD", String.format(Locale.ROOT,
+                                             "compressed %d to %d bytes (%.2f%%) for deletes/updates; private segment %s",
+                                             updates.bytesUsed.get(), bytesUsed, 100.*bytesUsed/updates.bytesUsed.get(),
+                                             privateSegment));
+    }
+  }
+
+  private byte[] freezeNumericDVUpdates(Map<String,LinkedHashMap<Term,NumericDocValuesUpdate>> numericDVUpdates)
+    throws IOException {
+    // TODO: we could do better here, e.g. collate the updates by field
+    // so if you are updating 2 fields interleaved we don't keep writing the field strings
+
+    RAMOutputStream out = new RAMOutputStream();
+    String lastTermField = null;
+    String lastUpdateField = null;
+    for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : numericDVUpdates.values()) {
+      numericDVUpdateCount += numericUpdates.size();
+      for (NumericDocValuesUpdate update : numericUpdates.values()) {
+
+        int code = update.term.bytes().length << 2;
+
+        String termField = update.term.field();
+        if (termField.equals(lastTermField) == false) {
+          code |= 1;
+        }
+        String updateField = update.field;
+        if (updateField.equals(lastUpdateField) == false) {
+          code |= 2;
+        }
+        out.writeVInt(code);
+        out.writeVInt(update.docIDUpto);
+        if ((code & 1) != 0) {
+          out.writeString(termField);
+          lastTermField = termField;
+        }
+        if ((code & 2) != 0) {
+          out.writeString(updateField);
+          lastUpdateField = updateField;
+        }
+
+        out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
+        out.writeZLong(((Long) update.value).longValue());
+      }
+    }
+    byte[] bytes = new byte[(int) out.getFilePointer()];
+    out.writeTo(bytes, 0);
+    return bytes;
+  }
+
+  private byte[] freezeBinaryDVUpdates(Map<String,LinkedHashMap<Term,BinaryDocValuesUpdate>> binaryDVUpdates)
+    throws IOException {
+    // TODO: we could do better here, e.g. collate the updates by field
+    // so if you are updating 2 fields interleaved we don't keep writing the field strings
+
+    RAMOutputStream out = new RAMOutputStream();
+    String lastTermField = null;
+    String lastUpdateField = null;
+    for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : binaryDVUpdates.values()) {
+      binaryDVUpdateCount += binaryUpdates.size();
       for (BinaryDocValuesUpdate update : binaryUpdates.values()) {
-        allBinaryUpdates.add(update);
-        binaryUpdatesSize += update.sizeInBytes();
+
+        int code = update.term.bytes().length << 2;
+
+        String termField = update.term.field();
+        if (termField.equals(lastTermField) == false) {
+          code |= 1;
+        }
+        String updateField = update.field;
+        if (updateField.equals(lastUpdateField) == false) {
+          code |= 2;
+        }
+        out.writeVInt(code);
+        out.writeVInt(update.docIDUpto);
+        if (termField.equals(lastTermField) == false) {
+          out.writeString(termField);
+          lastTermField = termField;
+        }
+        if (updateField.equals(lastUpdateField) == false) {
+          out.writeString(updateField);
+          lastUpdateField = updateField;
+        }
+        out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
+
+        BytesRef value = (BytesRef) update.value;
+        out.writeVInt(value.length);
+        out.writeBytes(value.bytes, value.offset, value.length);
       }
     }
-    binaryDVUpdates = allBinaryUpdates.toArray(new BinaryDocValuesUpdate[allBinaryUpdates.size()]);
-    
-    bytesUsed = (int) (terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY 
-        + numericUpdatesSize + RamUsageEstimator.shallowSizeOf(numericDVUpdates)
-        + binaryUpdatesSize + RamUsageEstimator.shallowSizeOf(binaryDVUpdates));
-    
-    numTermDeletes = deletes.numTermDeletes.get();
+    byte[] bytes = new byte[(int) out.getFilePointer()];
+    out.writeTo(bytes, 0);
+    return bytes;
   }
-  
-  public void setDelGen(long gen) {
-    assert this.gen == -1;
-    this.gen = gen;
-    terms.setDelGen(gen);
+
+  /** Returns the {@link SegmentCommitInfo} that this packet is supposed to apply its deletes to, or null
+   *  if the private segment was already merged away. */
+  private List<SegmentCommitInfo> getInfosToApply(IndexWriter writer) {
+    assert Thread.holdsLock(writer);
+    List<SegmentCommitInfo> infos;
+    if (privateSegment != null) {
+      if (writer.segmentInfos.indexOf(privateSegment) == -1) {
+        if (infoStream.isEnabled("BD")) {
+          infoStream.message("BD", "private segment already gone; skip processing updates");
+        }
+        return null;
+      } else {
+        infos = Collections.singletonList(privateSegment);
+      }
+    } else {
+      infos = writer.segmentInfos.asList();
+    }
+    return infos;
   }
-  
-  public long delGen() {
-    assert gen != -1;
-    return gen;
+
+  /** Translates a frozen packet of delete term/query, or doc values
+   *  updates, into their actual docIDs in the index, and applies the change.  This is a heavy
+   *  operation and is done concurrently by incoming indexing threads. */
+
+  public synchronized void apply(IndexWriter writer) throws IOException {
+    if (applied.getCount() == 0) {
+      // already done
+      return;
+    }
+
+    long startNS = System.nanoTime();
+
+    assert any();
+
+    Set<SegmentCommitInfo> seenSegments = new HashSet<>();
+
+    int iter = 0;
+    int totalSegmentCount = 0;
+    long totalDelCount = 0;
+
+    // Optimistic concurrency: assume we are free to resolve the deletes against all current segments in the index, despite that
+    // concurrent merges are running.  Once we are done, we check to see if a merge completed while we were running.  If so, we must retry
+    // resolving against the newly merged segment(s).  Eventually no merge finishes while we were running and we are done.
+    while (true) {
+      String messagePrefix;
+      if (iter == 0) {
+        messagePrefix = "";
+      } else {
+        messagePrefix = "iter " + iter;
+      }
+
+      long iterStartNS = System.nanoTime();
+
+      long mergeGenStart = writer.mergeFinishedGen.get();
+
+      Set<String> delFiles = new HashSet<>();
+      BufferedUpdatesStream.SegmentState[] segStates;
+
+      synchronized (writer) {
+        List<SegmentCommitInfo> infos = getInfosToApply(writer);
+        if (infos == null) {
+          break;
+        }
+
+        for (SegmentCommitInfo info : infos) {
+          delFiles.addAll(info.files());
+        }
+
+        // Must open while holding IW lock so that e.g. segments are not merged
+        // away, dropped from 100% deletions, etc., before we can open the readers
+        segStates = writer.bufferedUpdatesStream.openSegmentStates(writer.readerPool, infos, seenSegments, delGen());
+
+        if (segStates.length == 0) {
+
+          if (infoStream.isEnabled("BD")) {
+            infoStream.message("BD", "packet matches no segments");
+          }
+          
+          break;
+        }
+
+        if (infoStream.isEnabled("BD")) {
+          infoStream.message("BD", String.format(Locale.ROOT,
+                                                 messagePrefix + "now apply del packet (%s) to %d segments, mergeGen %d",
+                                                 this, segStates.length, mergeGenStart));
+        }
+
+        totalSegmentCount += segStates.length;
+
+        // Important, else IFD may try to delete our files while we are still using them,
+        // if e.g. a merge finishes on some of the segments we are resolving on:
+        writer.deleter.incRef(delFiles);
+      }
+
+      boolean success = false;
+      long delCount;
+      try {
+        // don't hold IW monitor lock here so threads are free concurrently resolve deletes/updates:
+        delCount = apply(segStates);
+        success = true;
+      } finally {
+        finishApply(writer, segStates, success, delFiles);
+      }
+
+      // Since we jus resolved some more deletes/updates, now is a good time to write them:
+      writer.readerPool.writeSomeDocValuesUpdates();
+
+      // It's OK to add this here, even if the while loop retries, because delCount only includes newly
+      // deleted documents, on the segments we didn't already do in previous iterations:
+      totalDelCount += delCount;
+
+      if (infoStream.isEnabled("BD")) {
+        infoStream.message("BD", String.format(Locale.ROOT,
+                                               messagePrefix + "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
+                                               this, segStates.length, delCount, (System.nanoTime() - iterStartNS) / 1000000000.));
+      }
+      
+      if (privateSegment != null) {
+        // No need to retry for a segment-private packet: the merge that folds in our private segment already waits for all deletes to
+        // be applied before it kicks off, so this private segment must already not be in the set of merging segments
+
+        break;
+      }
+
+      // Must sync on writer here so that IW.mergeCommit is not running concurrently, so that if we exit, we know mergeCommit will succeed
+      // in pulling all our delGens into a merge:
+      synchronized (writer) {
+        long mergeGenCur = writer.mergeFinishedGen.get();
+
+        if (mergeGenCur == mergeGenStart) {
+
+          // No merge finished while we were applying, so we are done!
+          break;
+        }
+      }
+
+      if (infoStream.isEnabled("BD")) {
+        infoStream.message("BD", messagePrefix + "concurrent merges finished; move to next iter");
+      }
+        
+      // A merge completed while we were running.  In this case, that merge may have picked up some of the updates we did, but not
+      // necessarily all of them, so we cycle again, re-applying all our updates to the newly merged segment.
+
+      iter++;
+    }
+
+    // Record that this packet is finished:
+    writer.bufferedUpdatesStream.finished(this);
+
+    if (infoStream.isEnabled("BD")) {
+      String message = String.format(Locale.ROOT,
+                                     "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
+                                     this, totalSegmentCount, totalDelCount, (System.nanoTime() - startNS) / 1000000000.);
+      if (iter > 0) {
+        message += "; " + (iter+1) + " iters due to concurrent merges";
+      }
+      message += "; " + writer.bufferedUpdatesStream.getPendingUpdatesCount() + " packets remain";
+      infoStream.message("BD", message);
+    }
+  }
+
+  private void finishApply(IndexWriter writer, BufferedUpdatesStream.SegmentState[] segStates,
+                           boolean success, Set<String> delFiles) throws IOException {
+    synchronized (writer) {
+
+      BufferedUpdatesStream.ApplyDeletesResult result;
+      try {
+        result = writer.bufferedUpdatesStream.closeSegmentStates(writer.readerPool, segStates, success);
+      } finally {
+        // Matches the incRef we did above, but we must do the decRef after closing segment states else
+        // IFD can't delete still-open files
+        writer.deleter.decRef(delFiles);
+      }
+
+      if (result.anyDeletes) {
+        writer.maybeMerge.set(true);
+        writer.checkpoint();
+      }
+
+      if (writer.keepFullyDeletedSegments == false && result.allDeleted != null) {
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "drop 100% deleted segments: " + writer.segString(result.allDeleted));
+        }
+        for (SegmentCommitInfo info : result.allDeleted) {
+          writer.dropDeletedSegment(info);
+        }
+        writer.checkpoint();
+      }
+    }
+  }
+
+  /** Applies pending delete-by-term, delete-by-query and doc values updates to all segments in the index, returning
+   *  the number of new deleted or updated documents. */
+  private synchronized long apply(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
+
+    if (delGen == -1) {
+      // we were not yet pushed
+      throw new IllegalArgumentException("gen is not yet set; call BufferedUpdatesStream.push first");
+    }
+
+    if (applied.getCount() == 0) {
+      // already done
+      return totalDelCount;
+    }
+
+    if (privateSegment != null) {
+      assert segStates.length == 1;
+      assert privateSegment == segStates[0].reader.getSegmentInfo();
+    }
+
+    totalDelCount += applyTermDeletes(segStates);
+    totalDelCount += applyQueryDeletes(segStates);
+    totalDelCount += applyDocValuesUpdates(segStates);
+
+    return totalDelCount;
   }
 
-  public TermIterator termIterator() {
-    return terms.iterator();
+  private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
+
+    if (numericDVUpdates.length == 0 && binaryDVUpdates.length == 0) {
+      return 0;
+    }
+
+    long startNS = System.nanoTime();
+
+    long updateCount = 0;
+
+    for (BufferedUpdatesStream.SegmentState segState : segStates) {
+
+      if (delGen < segState.delGen) {
+        // segment is newer than this deletes packet
+        continue;
+      }
+
+      if (segState.rld.refCount() == 1) {
+        // This means we are the only remaining reference to this segment, meaning
+        // it was merged away while we were running, so we can safely skip running
+        // because we will run on the newly merged segment next:
+        continue;
+      }
+
+      if (numericDVUpdates.length > 0) {
+        updateCount += applyDocValuesUpdates(segState, numericDVUpdates, true);
+      }
+
+      if (binaryDVUpdates.length > 0) {
+        updateCount += applyDocValuesUpdates(segState, binaryDVUpdates, false);
+      }
+    }
+
+    if (infoStream.isEnabled("BD")) {
+      infoStream.message("BD",
+                         String.format(Locale.ROOT, "applyDocValuesUpdates %.1f msec for %d segments, %d numeric updates and %d binary updates; %d new updates",
+                                       (System.nanoTime()-startNS)/1000000.,
+                                       segStates.length,
+                                       numericDVUpdateCount,
+                                       binaryDVUpdateCount,
+                                       updateCount));
+    }
+
+    return updateCount;
+  }
+
+  private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState segState,
+                                     byte[] updates, boolean isNumeric) throws IOException {
+
+    TermsEnum termsEnum = null;
+    PostingsEnum postingsEnum = null;
+
+    // TODO: we can process the updates per DV field, from last to first so that
+    // if multiple terms affect same document for the same field, we add an update
+    // only once (that of the last term). To do that, we can keep a bitset which
+    // marks which documents have already been updated. So e.g. if term T1
+    // updates doc 7, and then we process term T2 and it updates doc 7 as well,
+    // we don't apply the update since we know T1 came last and therefore wins
+    // the update.
+    // We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
+    // that these documents aren't even returned.
+
+    long updateCount = 0;
+
+    // We first write all our updates private, and only in the end publish to the ReadersAndUpdates */
+    Map<String, DocValuesFieldUpdates> holder = new HashMap<>();
+
+    ByteArrayDataInput in = new ByteArrayDataInput(updates);
+
+    String termField = null;
+    String updateField = null;
+    BytesRef term = new BytesRef();
+    term.bytes = new byte[16];
+    
+    BytesRef scratch = new BytesRef();
+    scratch.bytes = new byte[16];
+    
+    while (in.getPosition() != updates.length) {
+      int code = in.readVInt();
+      int docIDUpto = in.readVInt();
+      term.length = code >> 2;
+      
+      if ((code & 1) != 0) {
+        termField = in.readString();
+      }
+      if ((code & 2) != 0) {
+        updateField = in.readString();
+      }
+
+      if (term.bytes.length < term.length) {
+        term.bytes = ArrayUtil.grow(term.bytes, term.length);
+      }
+      in.readBytes(term.bytes, 0, term.length);
+
+      int limit;
+      if (delGen == segState.delGen) {
+        assert privateSegment != null;
+        limit = docIDUpto;
+      } else {
+        limit = Integer.MAX_VALUE;
+      }
+        
+      // TODO: we traverse the terms in update order (not term order) so that we
+      // apply the updates in the correct order, i.e. if two terms udpate the
+      // same document, the last one that came in wins, irrespective of the
+      // terms lexical order.
+      // we can apply the updates in terms order if we keep an updatesGen (and
+      // increment it with every update) and attach it to each NumericUpdate. Note
+      // that we cannot rely only on docIDUpto because an app may send two updates
+      // which will get same docIDUpto, yet will still need to respect the order
+      // those updates arrived.
+
+      // TODO: we could at least *collate* by field?
+
+      // This is the field used to resolve to docIDs, e.g. an "id" field, not the doc values field we are updating!
+      if ((code & 1) != 0) {
+        Terms terms = segState.reader.terms(termField);
+        if (terms != null) {
+          termsEnum = terms.iterator();
+        } else {
+          termsEnum = null;
+        }
+      }
+
+      // TODO: can we avoid boxing here w/o fully forking this method?
+      Object value;
+      if (isNumeric) {
+        value = Long.valueOf(in.readZLong());
+      } else {
+        value = scratch;
+        scratch.length = in.readVInt();
+        if (scratch.bytes.length < scratch.length) {
+          scratch.bytes = ArrayUtil.grow(scratch.bytes, scratch.length);
+        }
+        in.readBytes(scratch.bytes, 0, scratch.length);
+      }
+
+      if (termsEnum == null) {
+        // no terms in this segment for this field
+        continue;
+      }
+
+      if (termsEnum.seekExact(term)) {
+
+        // we don't need term frequencies for this
+        postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+
+        DocValuesFieldUpdates dvUpdates = holder.get(updateField);
+        if (dvUpdates == null) {
+          if (isNumeric) {
+            dvUpdates = new NumericDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
+          } else {
+            dvUpdates = new BinaryDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
+          }
+
+          holder.put(updateField, dvUpdates);
+        }
+
+        if (segState.rld.sortMap != null && privateSegment != null) {
+          // This segment was sorted on flush; we must apply seg-private deletes carefully in this case:
+          int doc;
+          final Bits acceptDocs = segState.rld.getLiveDocs();
+          while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+            if (acceptDocs != null && acceptDocs.get(doc) == false) {
+              continue;
+            }
+            
+            // The limit is in the pre-sorted doc space:
+            if (segState.rld.sortMap.newToOld(doc) < limit) {
+              dvUpdates.add(doc, value);
+              updateCount++;
+            }
+          }
+        } else {
+          int doc;
+          final Bits acceptDocs = segState.rld.getLiveDocs();
+          while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+            if (doc >= limit) {
+              break; // no more docs that can be updated for this term
+            }
+            if (acceptDocs != null && acceptDocs.get(doc) == false) {
+              continue;
+            }
+            dvUpdates.add(doc, value);
+            updateCount++;
+          }
+        }
+      }
+    }
+
+    // now freeze & publish:
+    for (DocValuesFieldUpdates update : holder.values()) {
+      if (update.any()) {
+        update.finish();
+        segState.rld.addDVUpdate(update);
+      }
+    }
+
+    return updateCount;
   }
 
-  public Iterable<QueryAndLimit> queriesIterable() {
-    return new Iterable<QueryAndLimit>() {
-      @Override
-      public Iterator<QueryAndLimit> iterator() {
-        return new Iterator<QueryAndLimit>() {
-          private int upto;
+  // Delete by query
+  private long applyQueryDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
+
+    if (deleteQueries.length == 0) {
+      return 0;
+    }
+
+    long startNS = System.nanoTime();
+
+    long delCount = 0;
+    for (BufferedUpdatesStream.SegmentState segState : segStates) {
+
+      if (delGen < segState.delGen) {
+        // segment is newer than this deletes packet
+        continue;
+      }
+      
+      if (segState.rld.refCount() == 1) {
+        // This means we are the only remaining reference to this segment, meaning
+        // it was merged away while we were running, so we can safely skip running
+        // because we will run on the newly merged segment next:
+        continue;
+      }
 
-          @Override
-          public boolean hasNext() {
-            return upto < queries.length;
+      final LeafReaderContext readerContext = segState.reader.getContext();
+      for (int i = 0; i < deleteQueries.length; i++) {
+        Query query = deleteQueries[i];
+        int limit;
+        if (delGen == segState.delGen) {
+          assert privateSegment != null;
+          limit = deleteQueryLimits[i];
+        } else {
+          limit = Integer.MAX_VALUE;
+        }
+        final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
+        searcher.setQueryCache(null);
+        final Weight weight = searcher.createNormalizedWeight(query, false);
+        final Scorer scorer = weight.scorer(readerContext);
+        if (scorer != null) {
+          final DocIdSetIterator it = scorer.iterator();
+
+          int docID;
+          while ((docID = it.nextDoc()) < limit)  {
+            if (segState.rld.delete(docID)) {
+              delCount++;
+            }
           }
+        }
+      }
+    }
+
+    if (infoStream.isEnabled("BD")) {
+      infoStream.message("BD",
+                         String.format(Locale.ROOT, "applyQueryDeletes took %.2f msec for %d segments and %d queries; %d new deletions",
+                                       (System.nanoTime()-startNS)/1000000.,
+                                       segStates.length,
+                                       deleteQueries.length,
+                                       delCount));
+    }
+    
+    return delCount;
+  }
+  
+  private long applyTermDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
+
+    if (deleteTerms.size() == 0) {
+      return 0;
+    }
+
+    // We apply segment-private deletes on flush:
+    assert privateSegment == null;
+
+    try {
+      long startNS = System.nanoTime();
 
-          @Override
-          public QueryAndLimit next() {
-            QueryAndLimit ret = new QueryAndLimit(queries[upto], queryLimits[upto]);
-            upto++;
-            return ret;
+      long delCount = 0;
+
+      for (BufferedUpdatesStream.SegmentState segState : segStates) {
+        assert segState.delGen != delGen: "segState.delGen=" + segState.delGen + " vs this.gen=" + delGen;
+        if (segState.delGen > delGen) {
+          // our deletes don't apply to this segment
+          continue;
+        }
+        if (segState.rld.refCount() == 1) {
+          // This means we are the only remaining reference to this segment, meaning
+          // it was merged away while we were running, so we can safely skip running
+          // because we will run on the newly merged segment next:
+          continue;
+        }
+
+        FieldTermIterator iter = deleteTerms.iterator();
+
+        BytesRef delTerm;
+        String field = null;
+        TermsEnum termsEnum = null;
+        BytesRef readerTerm = null;
+        PostingsEnum postingsEnum = null;
+        while ((delTerm = iter.next()) != null) {
+
+          if (iter.field() != field) {
+            // field changed
+            field = iter.field();
+            Terms terms = segState.reader.terms(field);
+            if (terms != null) {
+              termsEnum = terms.iterator();
+              readerTerm = termsEnum.next();
+            } else {
+              termsEnum = null;
+            }
           }
 
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
+          if (termsEnum != null) {
+            int cmp = delTerm.compareTo(readerTerm);
+            if (cmp < 0) {
+              // TODO: can we advance across del terms here?
+              // move to next del term
+              continue;
+            } else if (cmp == 0) {
+              // fall through
+            } else if (cmp > 0) {
+              TermsEnum.SeekStatus status = termsEnum.seekCeil(delTerm);
+              if (status == TermsEnum.SeekStatus.FOUND) {
+                // fall through
+              } else if (status == TermsEnum.SeekStatus.NOT_FOUND) {
+                readerTerm = termsEnum.term();
+                continue;
+              } else {
+                // TODO: can we advance to next field in deleted terms?
+                // no more terms in this segment
+                termsEnum = null;
+                continue;
+              }
+            }
+
+            // we don't need term frequencies for this
+            postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+
+            assert postingsEnum != null;
+
+            int docID;
+            while ((docID = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+              // NOTE: there is no limit check on the docID
+              // when deleting by Term (unlike by Query)
+              // because on flush we apply all Term deletes to
+              // each segment.  So all Term deleting here is
+              // against prior segments:
+              if (segState.rld.delete(docID)) {
+                delCount++;
+              }
+            }
           }
-        };
+        }
+      }
+
+      if (infoStream.isEnabled("BD")) {
+        infoStream.message("BD",
+                           String.format(Locale.ROOT, "applyTermDeletes took %.2f msec for %d segments and %d del terms; %d new deletions",
+                                         (System.nanoTime()-startNS)/1000000.,
+                                         segStates.length,
+                                         deleteTerms.size(),
+                                         delCount));
       }
-    };
+
+      return delCount;
+
+    } catch (Throwable t) {
+      throw IOUtils.rethrowAlways(t);
+    }
+  }
+  
+  public void setDelGen(long delGen) {
+    assert this.delGen == -1: "delGen was already previously set to " + this.delGen;
+    this.delGen = delGen;
+    deleteTerms.setDelGen(delGen);
+  }
+  
+  public long delGen() {
+    assert delGen != -1;
+    return delGen;
   }
 
   @Override
   public String toString() {
-    String s = "";
+    String s = "delGen=" + delGen;
     if (numTermDeletes != 0) {
-      s += " " + numTermDeletes + " deleted terms (unique count=" + terms.size() + ")";
+      s += " numDeleteTerms=" + numTermDeletes;
+      if (numTermDeletes != deleteTerms.size()) {
+        s += " (" + deleteTerms.size() + " unique)";
+      }
+    }
+    if (deleteQueries.length != 0) {
+      s += " numDeleteQuerys=" + deleteQueries.length;
+    }
+    if (numericDVUpdates.length > 0) {
+      s += " numNumericDVUpdates=" + numericDVUpdateCount;
     }
-    if (queries.length != 0) {
-      s += " " + queries.length + " deleted queries";
+    if (binaryDVUpdates.length > 0) {
+      s += " numBinaryDVUpdates=" + binaryDVUpdateCount;
     }
     if (bytesUsed != 0) {
       s += " bytesUsed=" + bytesUsed;
     }
+    if (privateSegment != null) {
+      s += " privateSegment=" + privateSegment;
+    }
 
     return s;
   }
   
   boolean any() {
-    return terms.size() > 0 || queries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
+    return deleteTerms.size() > 0 || deleteQueries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
+  }
+
+  boolean anyDeleteTerms() {
+    return deleteTerms.size() > 0;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
index f7f196d..e2b822b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
@@ -510,9 +510,8 @@ final class IndexFileDeleter implements Closeable {
     assert locked();
 
     assert Thread.holdsLock(writer);
-    long t0 = 0;
+    long t0 = System.nanoTime();
     if (infoStream.isEnabled("IFD")) {
-      t0 = System.nanoTime();
       infoStream.message("IFD", "now checkpoint \"" + writer.segString(writer.toLiveInfos(segmentInfos)) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
     }
 
@@ -698,7 +697,9 @@ final class IndexFileDeleter implements Closeable {
     ensureOpen();
 
     if (infoStream.isEnabled("IFD")) {
-      infoStream.message("IFD", "delete " + names + "");
+      if (names.size() > 0) {
+        infoStream.message("IFD", "delete " + names + "");
+      }
     }
 
     // We make two passes, first deleting any segments_N files, second deleting the rest.  We do this so that if we throw exc or JVM