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 2016/06/11 15:49:33 UTC

[17/21] lucene-solr:branch_6x: LUCENE-6766: resolve remaining nocommits; add more IW infoStream logging during merge

LUCENE-6766: resolve remaining nocommits; add more IW infoStream logging during merge


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2703b827
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2703b827
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2703b827

Branch: refs/heads/branch_6x
Commit: 2703b827bf2316e8d39025666ed5f1d42ed70d64
Parents: a4722be
Author: Mike McCandless <mi...@apache.org>
Authored: Tue May 10 04:49:33 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Jun 11 11:48:40 2016 -0400

----------------------------------------------------------------------
 .../simpletext/SimpleTextSegmentInfoFormat.java |   2 -
 .../apache/lucene/codecs/TermVectorsWriter.java |   1 -
 .../lucene/codecs/lucene62/Lucene62Codec.java   |   2 -
 .../lucene62/Lucene62SegmentInfoFormat.java     |   6 +-
 .../org/apache/lucene/index/DocIDMerger.java    |  17 +-
 .../java/org/apache/lucene/index/DocValues.java |   2 +-
 .../org/apache/lucene/index/IndexWriter.java    |  22 +-
 .../org/apache/lucene/index/MergeState.java     |  23 +-
 .../apache/lucene/index/MultiPostingsEnum.java  |   2 -
 .../org/apache/lucene/index/MultiSorter.java    |   2 +-
 .../org/apache/lucene/index/SegmentMerger.java  |   2 -
 .../org/apache/lucene/index/TestAddIndexes.java |  51 +++++
 .../apache/lucene/index/TestDocIDMerger.java    |  38 +++-
 .../apache/lucene/index/TestIndexSorting.java   |  86 +++++--
 .../TestEarlyTerminatingSortingCollector.java   |   2 -
 .../search/BlockJoinComparatorSource.java       | 225 -------------------
 .../lucene/index/TestBlockJoinSorter.java       | 128 -----------
 17 files changed, 196 insertions(+), 415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
index 8ab45be..146e92a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
@@ -242,7 +242,6 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
                 break;
             }
             break;
-          // nocommit need the rest
           default:
             throw new AssertionError();
         }
@@ -370,7 +369,6 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
           case FLOAT:
             sortType = "float";
             break;
-          // nocommit the rest:
           default:
             throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
index 81dd095..5756d5b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
@@ -203,7 +203,6 @@ public abstract class TermVectorsWriter implements Closeable {
       if (reader != null) {
         reader.checkIntegrity();
       }
-      // nocommit make sure the else case tested here
       subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], reader, mergeState.maxDocs[i]));
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
index aa0adae..5071075 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java
@@ -41,8 +41,6 @@ import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
-// nocommit if somehow this does NOT land in 6.2, rename all this!!
-
 /**
  * Implements the Lucene 6.2 index format, with configurable per-field postings
  * and docvalues formats.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
index da19594..fe78572 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java
@@ -37,8 +37,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Version;
 
-// nocommit fixup javadocs here:
-
 /**
  * Lucene 6.2 Segment info format.
  * <p>
@@ -54,7 +52,9 @@ import org.apache.lucene.util.Version;
  *   <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
  *   <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
  *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
- *   <li>IndexSort --&gt; {@link DataOutput#writeInt Int32} count, followed by {@code count} SortField</li>
+ *   <li>IndexSort --&gt; {@link DataOutput#writeVInt Int32} count, followed by {@code count} SortField</li>
+ *   <li>SortField --&gt; {@link DataOutput#writeString String} field name, followed by {@link DataOutput#writeVInt Int32} sort type ID,
+ *       followed by {@link DataOutput#writeByte Int8} indicatating reversed sort, followed by a type-specific encoding of the optional missing value
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * Field Descriptions:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
index 7876ebc..e8ffc6c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -58,7 +58,6 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
     this.subs = subs;
 
     if (indexIsSorted) {
-      // nocommit: we could optimize the (silly) single-sub case and pretend it's a concatenation instead
       queue = new PriorityQueue<T>(maxCount) {
         @Override
         protected boolean lessThan(Sub a, Sub b) {
@@ -74,7 +73,6 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
     reset();
   }
 
-  // nocommit it's awkward that we must pass in this boolean, when the subs should "know" this based on what docMap they have?
   public DocIDMerger(List<T> subs, boolean indexIsSorted) {
     this(subs, subs.size(), indexIsSorted);
   }
@@ -82,7 +80,8 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
   /** Reuse API, currently only used by postings during merge */
   public void reset() {
     if (queue != null) {
-      assert queue.size() == 0;
+      // caller may not have fully consumed the queue:
+      queue.clear();
       for(T sub : subs) {
         while (true) {
           int docID = sub.nextDoc();
@@ -103,14 +102,12 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
         }
       }
       first = true;
+    } else if (subs.size() > 0) {
+      current = subs.get(0);
+      nextIndex = 1;
     } else {
-      if (subs.size() > 0) {
-        current = subs.get(0);
-        nextIndex = 1;
-      } else {
-        current = null;
-        nextIndex = 0;
-      }
+      current = null;
+      nextIndex = 0;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/DocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValues.java b/lucene/core/src/java/org/apache/lucene/index/DocValues.java
index feceb3b..15b15c6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValues.java
@@ -210,7 +210,7 @@ public final class DocValues {
                                         (expected.length == 1 
                                         ? "(expected=" + expected[0]
                                         : "(expected one of " + Arrays.toString(expected)) + "). " +
-                                        "Use UninvertingReader or index with docvalues.");
+                                        " Re-index with correct docvalues type.");
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
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 0289c61..159f591 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -16,8 +16,6 @@
  */
 package org.apache.lucene.index;
 
-// nocommit must add sorted indices to back compat tests
-
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -2490,9 +2488,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    * @throws IllegalArgumentException if addIndexes would cause
-   *   the index to exceed {@link #MAX_DOCS}
+   *   the index to exceed {@link #MAX_DOCS}, or if the indoming
+   *   index sort does not match this index's index sort
    */
-  // nocommit doesn't support index sorting?  or sorts must be the same?
   public void addIndexes(Directory... dirs) throws IOException {
     ensureOpen();
 
@@ -2500,6 +2498,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
 
     List<Lock> locks = acquireWriteLocks(dirs);
 
+    Sort indexSort = config.getIndexSort();
+
     boolean successTop = false;
 
     try {
@@ -2532,6 +2532,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           for (SegmentCommitInfo info : sis) {
             assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
 
+            Sort segmentIndexSort = info.info.getIndexSort();
+
+            if (indexSort != null && segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
+              // TODO: we could make this smarter, e.g. if the incoming indexSort is congruent with our sort ("starts with") then it's OK
+              throw new IllegalArgumentException("cannot change index sort from " + segmentIndexSort + " to " + indexSort);
+            }
+
             String newSegName = newSegmentName();
 
             if (infoStream.isEnabled("IW")) {
@@ -2622,13 +2629,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * @throws IllegalArgumentException
    *           if addIndexes would cause the index to exceed {@link #MAX_DOCS}
    */
-  // nocommit make sure if you add "sorted by X" to "sorted by Y" index, we catch it
   public void addIndexes(CodecReader... readers) throws IOException {
     ensureOpen();
 
     // long so we can detect int overflow:
     long numDocs = 0;
 
+    Sort indexSort = config.getIndexSort();
+
     try {
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "flush at addIndexes(CodecReader...)");
@@ -2638,6 +2646,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       String mergedName = newSegmentName();
       for (CodecReader leaf : readers) {
         numDocs += leaf.numDocs();
+        Sort leafIndexSort = leaf.getIndexSort();
+        if (indexSort != null && leafIndexSort != null && indexSort.equals(leafIndexSort) == false) {
+          throw new IllegalArgumentException("cannot change index sort from " + leafIndexSort + " to " + indexSort);
+        }
       }
       
       // Best-effort up front check:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/MergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index 63eab98..7737ff2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -42,8 +42,8 @@ public class MergeState {
   /** Maps document IDs from old segments to document IDs in the new segment */
   public final DocMap[] docMaps;
 
-  // nocommit can we somehow not need to expose this?  should IW's reader pool always sort on load...?
-  public final DocMap[] leafDocMaps;
+  // Only used by IW when it must remap deletes that arrived against the merging segmetns while a merge was running:
+  final DocMap[] leafDocMaps;
 
   /** {@link SegmentInfo} of the newly merged segment. */
   public final SegmentInfo segmentInfo;
@@ -84,6 +84,8 @@ public class MergeState {
   /** Sole constructor. */
   MergeState(List<CodecReader> originalReaders, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
 
+    this.infoStream = infoStream;
+
     final Sort indexSort = segmentInfo.getIndexSort();
     int numReaders = originalReaders.size();
     leafDocMaps = new DocMap[numReaders];
@@ -138,7 +140,6 @@ public class MergeState {
     segmentInfo.setMaxDoc(numDocs);
 
     this.segmentInfo = segmentInfo;
-    this.infoStream = infoStream;
     this.docMaps = buildDocMaps(readers, indexSort);
   }
 
@@ -219,6 +220,9 @@ public class MergeState {
         // This segment was written by flush, so documents are not yet sorted, so we sort them now:
         Sorter.DocMap sortDocMap = sorter.sort(leaf);
         if (sortDocMap != null) {
+          if (infoStream.isEnabled("SM")) {
+            infoStream.message("SM", "segment " + leaf + " is not sorted; wrapping for sort " + indexSort + " now");
+          }
           leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(new MergeReaderWrapper(leaf), sortDocMap));
           leafDocMaps[readers.size()] = new DocMap() {
               @Override
@@ -226,10 +230,19 @@ public class MergeState {
                 return sortDocMap.oldToNew(docID);
               }
             };
+        } else {
+          if (infoStream.isEnabled("SM")) {
+            infoStream.message("SM", "segment " + leaf + " is not sorted, but is already accidentally in sort " + indexSort + " order");
+          }
         }
 
-      } else if (segmentSort.equals(indexSort) == false) {
-        throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort + " but to-be-merged segment has sort=" + segmentSort);
+      } else {
+        if (segmentSort.equals(indexSort) == false) {
+          throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort + " but to-be-merged segment has sort=" + segmentSort);
+        }
+        if (infoStream.isEnabled("SM")) {
+          infoStream.message("SM", "segment " + leaf + " already sorted");
+        }
       }
 
       readers.add(leaf);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java
index 573bbe8..42e3f41 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java
@@ -57,8 +57,6 @@ public final class MultiPostingsEnum extends PostingsEnum {
     return this.parent == parent;
   }
 
-  // nocommit is this class supposed to be aware of index sorting too???
-
   /** Re-use and reset this instance on the provided slices. */
   public MultiPostingsEnum reset(final EnumWithSlice[] subs, final int numSubs) {
     this.numSubs = numSubs;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
index 3448c90..6a5eb5a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
@@ -36,7 +36,7 @@ final class MultiSorter {
    *  documents into the merged segment.  The documents for each incoming leaf reader must already be sorted by the same sort! */
   static MergeState.DocMap[] sort(Sort sort, List<CodecReader> readers) throws IOException {
 
-    // nocommit optimize if only 1 reader is incoming
+    // TODO: optimize if only 1 reader is incoming, though that's a rare case
 
     SortField fields[] = sort.getSort();
     final CrossReaderComparator[] comparators = new CrossReaderComparator[fields.length];

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
index 0cc1823..d23f010 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
@@ -48,8 +48,6 @@ final class SegmentMerger {
   final MergeState mergeState;
   private final FieldInfos.Builder fieldInfosBuilder;
 
-  // nocommit make sure infoStream states per-segment-being-merged if they are already sorted
-
   // note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
   SegmentMerger(List<CodecReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
                 FieldInfos.FieldNumbers fieldNumbers, IOContext context) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
index a78af96..876328a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
@@ -39,6 +39,8 @@ import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
@@ -1281,4 +1283,53 @@ public class TestAddIndexes extends LuceneTestCase {
     w2.close();
     IOUtils.close(src, dest);
   }
+
+  public void testIllegalIndexSortChange1() throws Exception {
+    Directory dir1 = newDirectory();
+    IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
+    RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1, iwc1);
+    w1.addDocument(new Document());
+    w1.commit();
+    w1.addDocument(new Document());
+    w1.commit();
+    // so the index sort is in fact burned into the index:
+    w1.forceMerge(1);
+    w1.close();
+
+    Directory dir2 = newDirectory();
+    IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc2.setIndexSort(new Sort(new SortField("foo", SortField.Type.STRING)));
+    RandomIndexWriter w2 = new RandomIndexWriter(random(), dir2, iwc2);
+    String message = expectThrows(IllegalArgumentException.class, () -> {
+        w2.addIndexes(dir1);
+      }).getMessage();
+    assertEquals("cannot change index sort from <int: \"foo\"> to <string: \"foo\">", message);
+    IOUtils.close(dir1, w2, dir2);
+  }
+
+  public void testIllegalIndexSortChange2() throws Exception {
+    Directory dir1 = newDirectory();
+    IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc1.setIndexSort(new Sort(new SortField("foo", SortField.Type.INT)));
+    RandomIndexWriter w1 = new RandomIndexWriter(random(), dir1, iwc1);
+    w1.addDocument(new Document());
+    w1.commit();
+    w1.addDocument(new Document());
+    w1.commit();
+    // so the index sort is in fact burned into the index:
+    w1.forceMerge(1);
+    w1.close();
+
+    Directory dir2 = newDirectory();
+    IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc2.setIndexSort(new Sort(new SortField("foo", SortField.Type.STRING)));
+    RandomIndexWriter w2 = new RandomIndexWriter(random(), dir2, iwc2);
+    IndexReader r1 = DirectoryReader.open(dir1);
+    String message = expectThrows(IllegalArgumentException.class, () -> {
+        w2.addIndexes((SegmentReader) getOnlyLeafReader(r1));
+      }).getMessage();
+    assertEquals("cannot change index sort from <int: \"foo\"> to <string: \"foo\">", message);
+    IOUtils.close(r1, dir1, w2, dir2);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
index 949dece..003db9e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocIDMerger.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -147,19 +148,35 @@ public class TestDocIDMerger extends LuceneTestCase {
     }
     assertEquals(0, oldToNew.size());
 
+    // sometimes do some deletions:
+    final FixedBitSet liveDocs;
+    if (random().nextBoolean()) {
+      liveDocs = new FixedBitSet(totDocCount);
+      liveDocs.set(0, totDocCount);
+      int deleteAttemptCount = TestUtil.nextInt(random(), 1, totDocCount);
+      for(int i=0;i<deleteAttemptCount;i++) {
+        liveDocs.clear(random().nextInt(totDocCount));
+      }
+    } else {
+      liveDocs = null;
+    }
+
     List<TestSubSorted> subs = new ArrayList<>();
     for(int i=0;i<subCount;i++) {
       final int[] docMap = completedSubs.get(i);
       subs.add(new TestSubSorted(new MergeState.DocMap() {
           @Override
           public int get(int docID) {
-            return docMap[docID];
+            int mapped = docMap[docID];
+            if (liveDocs == null || liveDocs.get(mapped)) {
+              return mapped;
+            } else {
+              return -1;
+            }
           }
         }, docMap.length, i));
     }
 
-    // nocommit test w/ deletions too
-
     DocIDMerger<TestSubSorted> merger = new DocIDMerger<>(subs, true);
 
     int count = 0;
@@ -168,12 +185,21 @@ public class TestDocIDMerger extends LuceneTestCase {
       if (sub == null) {
         break;
       }
+      if (liveDocs != null) {
+        count = liveDocs.nextSetBit(count);
+      }
       assertEquals(count, sub.mappedDocID);
       count++;
     }
 
-    assertEquals(totDocCount, count);
+    if (liveDocs != null) {
+      if (count < totDocCount) {
+        assertEquals(NO_MORE_DOCS, liveDocs.nextSetBit(count));
+      } else {
+        assertEquals(totDocCount, count);
+      }
+    } else {
+      assertEquals(totDocCount, count);
+    }
   }
-
-  // nocommit more tests, e.g. deleted docs
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index ba17131..4deadd3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -47,6 +47,7 @@ import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
@@ -78,15 +79,6 @@ import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-// nocommit test tie break
-// nocommit test multiple sorts
-// nocommit test update dvs
-// nocommit test missing value
-
-// nocommit test EarlyTerminatingCollector
-
-// nocommit must test all supported SortField.Type
-
 public class TestIndexSorting extends LuceneTestCase {
 
   public void testBasicString() throws Exception {
@@ -881,8 +873,13 @@ public class TestIndexSorting extends LuceneTestCase {
     dir.close();
   }
 
-  public void testAddIndexes(boolean withDeletes) throws Exception {
+  public void testAddIndexes(boolean withDeletes, boolean useReaders) throws Exception {
     Directory dir = newDirectory();
+    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
+    IndexWriterConfig iwc1 = newIndexWriterConfig();
+    if (random().nextBoolean()) {
+      iwc1.setIndexSort(indexSort);
+    }
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     final int numDocs = atLeast(100);
     for (int i = 0; i < numDocs; ++i) {
@@ -896,19 +893,26 @@ public class TestIndexSorting extends LuceneTestCase {
         w.deleteDocuments(new Term("id", Integer.toString(i)));
       }
     }
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
     final IndexReader reader = w.getReader();
+    w.close();
 
     Directory dir2 = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
-    Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
     iwc.setIndexSort(indexSort);
     IndexWriter w2 = new IndexWriter(dir2, iwc);
 
-    CodecReader[] codecReaders = new CodecReader[reader.leaves().size()];
-    for (int i = 0; i < codecReaders.length; ++i) {
-      codecReaders[i] = (CodecReader) reader.leaves().get(i).reader();
+    if (useReaders) {
+      CodecReader[] codecReaders = new CodecReader[reader.leaves().size()];
+      for (int i = 0; i < codecReaders.length; ++i) {
+        codecReaders[i] = (CodecReader) reader.leaves().get(i).reader();
+      }
+      w2.addIndexes(codecReaders);
+    } else {
+      w2.addIndexes(dir);
     }
-    w2.addIndexes(codecReaders);
     final IndexReader reader2 = w2.getReader();
     final IndexSearcher searcher = newSearcher(reader);
     final IndexSearcher searcher2 = newSearcher(reader2);
@@ -924,15 +928,23 @@ public class TestIndexSorting extends LuceneTestCase {
       }
     }
 
-    IOUtils.close(reader, reader2, w, w2, dir, dir2);
+    IOUtils.close(reader, reader2, w2, dir, dir2);
   }
 
   public void testAddIndexes() throws Exception {
-    testAddIndexes(false);
+    testAddIndexes(false, true);
   }
 
   public void testAddIndexesWithDeletions() throws Exception {
-    testAddIndexes(true);
+    testAddIndexes(true, true);
+  }
+
+  public void testAddIndexesWithDirectory() throws Exception {
+    testAddIndexes(false, false);
+  }
+
+  public void testAddIndexesWithDeletionsAndDirectory() throws Exception {
+    testAddIndexes(true, false);
   }
 
   public void testBadSort() throws Exception {
@@ -1126,7 +1138,6 @@ public class TestIndexSorting extends LuceneTestCase {
     public final float floatValue;
     public final double doubleValue;
     public final byte[] bytesValue;
-    // nocommit postings, points, term vectors
 
     public RandomDoc(int id) {
       this.id = id;
@@ -1194,7 +1205,7 @@ public class TestIndexSorting extends LuceneTestCase {
     if (TEST_NIGHTLY) {
       numDocs = atLeast(100000);
     } else {
-      numDocs = atLeast(1000);
+      numDocs = atLeast(10000);
     }
     List<RandomDoc> docs = new ArrayList<>();
 
@@ -1309,4 +1320,39 @@ public class TestIndexSorting extends LuceneTestCase {
 
     IOUtils.close(r1, r2, w1, w2, dir1, dir2);
   }
+
+  public void testTieBreak() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setIndexSort(new Sort(new SortField("foo", SortField.Type.STRING)));
+    iwc.setMergePolicy(newLogMergePolicy());
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<1000;id++) {
+      Document doc = new Document();
+      doc.add(new StoredField("id", id));
+      String value;
+      if (id < 500) {
+        value = "bar2";
+      } else {
+        value = "bar1";
+      }
+      doc.add(new SortedDocValuesField("foo", new BytesRef(value)));
+      w.addDocument(doc);
+      if (id == 500) {
+        w.commit();
+      }
+    }
+    w.forceMerge(1);
+    DirectoryReader r = DirectoryReader.open(w);
+    for(int docID=0;docID<1000;docID++) {
+      int expectedID;
+      if (docID < 500) {
+        expectedID = 500 + docID;
+      } else {
+        expectedID = docID - 500;
+      }
+      assertEquals(expectedID, r.document(docID).getField("id").numericValue().intValue());
+    }
+    IOUtils.close(r, w, dir);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
index 6108992..84d326f 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java
@@ -90,8 +90,6 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
     }
     iwc.setMergeScheduler(new SerialMergeScheduler()); // for reproducible tests
     iwc.setIndexSort(sort);
-    // nocommit:
-    iwc.setCodec(Codec.forName("SimpleText"));
     iw = new RandomIndexWriter(new Random(seed), dir, iwc);
     iw.setDoRandomForceMerge(false); // don't do this, it may happen anyway with MockRandomMP
     for (int i = 0; i < numDocs; ++i) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java b/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
deleted file mode 100644
index 7633ff5..0000000
--- a/lucene/misc/src/java/org/apache/lucene/search/BlockJoinComparatorSource.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.util.BitSet;
-
-// nocommit what to do here?
-
-/**
- * Helper class to sort readers that contain blocks of documents.
- * <p>
- * Note that this class is intended to used with index sorting,
- * and for other purposes has some limitations:
- * <ul>
- *    <li>Cannot yet be used with {@link IndexSearcher#searchAfter(ScoreDoc, Query, int, Sort) IndexSearcher.searchAfter}
- *    <li>Filling sort field values is not yet supported.
- * </ul>
- * @lucene.experimental
- */
-// TODO: can/should we clean this thing up (e.g. return a proper sort value)
-// and move to the join/ module?
-public class BlockJoinComparatorSource extends FieldComparatorSource {
-  final Query parentsFilter;
-  final Sort parentSort;
-  final Sort childSort;
-
-  /**
-   * Create a new BlockJoinComparatorSource, sorting only blocks of documents
-   * with {@code parentSort} and not reordering children with a block.
-   *
-   * @param parentsFilter Filter identifying parent documents
-   * @param parentSort Sort for parent documents
-   */
-  public BlockJoinComparatorSource(Query parentsFilter, Sort parentSort) {
-    this(parentsFilter, parentSort, new Sort(SortField.FIELD_DOC));
-  }
-
-  /**
-   * Create a new BlockJoinComparatorSource, specifying the sort order for both
-   * blocks of documents and children within a block.
-   *
-   * @param parentsFilter Filter identifying parent documents
-   * @param parentSort Sort for parent documents
-   * @param childSort Sort for child documents in the same block
-   */
-  public BlockJoinComparatorSource(Query parentsFilter, Sort parentSort, Sort childSort) {
-    this.parentsFilter = parentsFilter;
-    this.parentSort = parentSort;
-    this.childSort = childSort;
-  }
-
-  @Override
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public FieldComparator<Integer> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
-    // we keep parallel slots: the parent ids and the child ids
-    final int parentSlots[] = new int[numHits];
-    final int childSlots[] = new int[numHits];
-
-    SortField parentFields[] = parentSort.getSort();
-    final int parentReverseMul[] = new int[parentFields.length];
-    final FieldComparator<?> parentComparators[] = new FieldComparator[parentFields.length];
-    for (int i = 0; i < parentFields.length; i++) {
-      parentReverseMul[i] = parentFields[i].getReverse() ? -1 : 1;
-      parentComparators[i] = parentFields[i].getComparator(1, i);
-    }
-
-    SortField childFields[] = childSort.getSort();
-    final int childReverseMul[] = new int[childFields.length];
-    final FieldComparator<?> childComparators[] = new FieldComparator[childFields.length];
-    for (int i = 0; i < childFields.length; i++) {
-      childReverseMul[i] = childFields[i].getReverse() ? -1 : 1;
-      childComparators[i] = childFields[i].getComparator(1, i);
-    }
-
-    // NOTE: we could return parent ID as value but really our sort "value" is more complex...
-    // So we throw UOE for now. At the moment you really should only use this at indexing time.
-    return new FieldComparator<Integer>() {
-      int bottomParent;
-      int bottomChild;
-      BitSet parentBits;
-      LeafFieldComparator[] parentLeafComparators;
-      LeafFieldComparator[] childLeafComparators;
-
-      @Override
-      public int compare(int slot1, int slot2) {
-        try {
-          return compare(childSlots[slot1], parentSlots[slot1], childSlots[slot2], parentSlots[slot2]);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-
-      @Override
-      public void setTopValue(Integer value) {
-        // we dont have enough information (the docid is needed)
-        throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
-      }
-
-      @Override
-      public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
-        if (parentBits != null) {
-          throw new IllegalStateException("This comparator can only be used on a single segment");
-        }
-        IndexSearcher searcher = new IndexSearcher(ReaderUtil.getTopLevelContext(context));
-        searcher.setQueryCache(null);
-        final Weight weight = searcher.createNormalizedWeight(parentsFilter, false);
-        final Scorer parents = weight.scorer(context);
-        if (parents == null) {
-          throw new IllegalStateException("LeafReader " + context.reader() + " contains no parents!");
-        }
-        parentBits = BitSet.of(parents.iterator(), context.reader().maxDoc());
-        parentLeafComparators = new LeafFieldComparator[parentComparators.length];
-        for (int i = 0; i < parentComparators.length; i++) {
-          parentLeafComparators[i] = parentComparators[i].getLeafComparator(context);
-        }
-        childLeafComparators = new LeafFieldComparator[childComparators.length];
-        for (int i = 0; i < childComparators.length; i++) {
-          childLeafComparators[i] = childComparators[i].getLeafComparator(context);
-        }
-
-        return new LeafFieldComparator() {
-
-          @Override
-          public int compareBottom(int doc) throws IOException {
-            return compare(bottomChild, bottomParent, doc, parent(doc));
-          }
-
-          @Override
-          public int compareTop(int doc) throws IOException {
-            // we dont have enough information (the docid is needed)
-            throw new UnsupportedOperationException("this comparator cannot be used with deep paging");
-          }
-
-          @Override
-          public void copy(int slot, int doc) throws IOException {
-            childSlots[slot] = doc;
-            parentSlots[slot] = parent(doc);
-          }
-
-          @Override
-          public void setBottom(int slot) {
-            bottomParent = parentSlots[slot];
-            bottomChild = childSlots[slot];
-          }
-
-          @Override
-          public void setScorer(Scorer scorer) {
-            for (LeafFieldComparator comp : parentLeafComparators) {
-              comp.setScorer(scorer);
-            }
-            for (LeafFieldComparator comp : childLeafComparators) {
-              comp.setScorer(scorer);
-            }
-          }
-
-        };
-      }
-
-      @Override
-      public Integer value(int slot) {
-        // really our sort "value" is more complex...
-        throw new UnsupportedOperationException("filling sort field values is not yet supported");
-      }
-
-      int parent(int doc) {
-        return parentBits.nextSetBit(doc);
-      }
-
-      int compare(int docID1, int parent1, int docID2, int parent2) throws IOException {
-        if (parent1 == parent2) { // both are in the same block
-          if (docID1 == parent1 || docID2 == parent2) {
-            // keep parents at the end of blocks
-            return docID1 - docID2;
-          } else {
-            return compare(docID1, docID2, childLeafComparators, childReverseMul);
-          }
-        } else {
-          int cmp = compare(parent1, parent2, parentLeafComparators, parentReverseMul);
-          if (cmp == 0) {
-            return parent1 - parent2;
-          } else {
-            return cmp;
-          }
-        }
-      }
-
-      int compare(int docID1, int docID2, LeafFieldComparator comparators[], int reverseMul[]) throws IOException {
-        for (int i = 0; i < comparators.length; i++) {
-          // TODO: would be better if copy() didnt cause a term lookup in TermOrdVal & co,
-          // the segments are always the same here...
-          comparators[i].copy(0, docID1);
-          comparators[i].setBottom(0);
-          int comp = reverseMul[i] * comparators[i].compareBottom(docID2);
-          if (comp != 0) {
-            return comp;
-          }
-        }
-        return 0; // no need to docid tiebreak
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    return "blockJoin(parentSort=" + parentSort + ",childSort=" + childSort + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2703b827/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java b/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
deleted file mode 100644
index 4a0d2b5..0000000
--- a/lucene/misc/src/test/org/apache/lucene/index/TestBlockJoinSorter.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-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.Store;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.search.BlockJoinComparatorSource;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.LuceneTestCase;
-
-public class TestBlockJoinSorter extends LuceneTestCase {
-
-  public void test() throws IOException {
-    final int numParents = atLeast(200);
-    IndexWriterConfig cfg = newIndexWriterConfig(new MockAnalyzer(random()));
-    cfg.setMergePolicy(newLogMergePolicy());
-    final RandomIndexWriter writer = new RandomIndexWriter(random(), newDirectory(), cfg);
-    final Document parentDoc = new Document();
-    final NumericDocValuesField parentVal = new NumericDocValuesField("parent_val", 0L);
-    parentDoc.add(parentVal);
-    final StringField parent = new StringField("parent", "true", Store.YES);
-    parentDoc.add(parent);
-    for (int i = 0; i < numParents; ++i) {
-      List<Document> documents = new ArrayList<>();
-      final int numChildren = random().nextInt(10);
-      for (int j = 0; j < numChildren; ++j) {
-        final Document childDoc = new Document();
-        childDoc.add(new NumericDocValuesField("child_val", random().nextInt(5)));
-        documents.add(childDoc);
-      }
-      parentVal.setLongValue(random().nextInt(50));
-      documents.add(parentDoc);
-      writer.addDocuments(documents);
-    }
-    writer.forceMerge(1);
-    IndexReader indexReader = writer.getReader();
-    writer.close();
-
-    IndexSearcher searcher = newSearcher(indexReader);
-    indexReader = searcher.getIndexReader(); // newSearcher may have wrapped it
-    assertEquals(1, indexReader.leaves().size());
-    final LeafReader reader = indexReader.leaves().get(0).reader();
-    final Query parentsFilter = new TermQuery(new Term("parent", "true"));
-
-    final Weight weight = searcher.createNormalizedWeight(parentsFilter, false);
-    final Scorer parents = weight.scorer(indexReader.leaves().get(0));
-    final BitSet parentBits = BitSet.of(parents.iterator(), reader.maxDoc());
-    final NumericDocValues parentValues = reader.getNumericDocValues("parent_val");
-    final NumericDocValues childValues = reader.getNumericDocValues("child_val");
-
-    final Sort parentSort = new Sort(new SortField("parent_val", SortField.Type.LONG));
-    final Sort childSort = new Sort(new SortField("child_val", SortField.Type.LONG));
-
-    final Sort sort = new Sort(new SortField("custom", new BlockJoinComparatorSource(parentsFilter, parentSort, childSort)));
-    final Sorter sorter = new Sorter(sort);
-    final Sorter.DocMap docMap = sorter.sort(reader);
-    assertEquals(reader.maxDoc(), docMap.size());
-
-    int[] children = new int[1];
-    int numChildren = 0;
-    int previousParent = -1;
-    for (int i = 0; i < docMap.size(); ++i) {
-      final int oldID = docMap.newToOld(i);
-      if (parentBits.get(oldID)) {
-        // check that we have the right children
-        for (int j = 0; j < numChildren; ++j) {
-          assertEquals(oldID, parentBits.nextSetBit(children[j]));
-        }
-        // check that children are sorted
-        for (int j = 1; j < numChildren; ++j) {
-          final int doc1 = children[j-1];
-          final int doc2 = children[j];
-          if (childValues.get(doc1) == childValues.get(doc2)) {
-            assertTrue(doc1 < doc2); // sort is stable
-          } else {
-            assertTrue(childValues.get(doc1) < childValues.get(doc2));
-          }
-        }
-        // check that parents are sorted
-        if (previousParent != -1) {
-          if (parentValues.get(previousParent) == parentValues.get(oldID)) {
-            assertTrue(previousParent < oldID);
-          } else {
-            assertTrue(parentValues.get(previousParent) < parentValues.get(oldID));
-          }
-        }
-        // reset
-        previousParent = oldID;
-        numChildren = 0;
-      } else {
-        children = ArrayUtil.grow(children, numChildren+1);
-        children[numChildren++] = oldID;
-      }
-    }
-    indexReader.close();
-    writer.w.getDirectory().close();
-  }
-
-}