Merging with trunk.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5650@1596756 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8526a8e..d0b3c32 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -185,6 +185,9 @@
 * LUCENE-5670: Add skip/FinalOutput to FST Outputs.  (Christian
   Ziech via Mike McCandless).
 
+* LUCENE-4236: Optimize BooleanQuery's in-order scoring. This speeds up
+  some types of boolean queries.  (Robert Muir)
+
 Bug fixes
 
 * LUCENE-5673: MMapDirectory: Work around a "bug" in the JDK that throws
@@ -200,6 +203,10 @@
   if the underlying TermsEnum supports ord() and the insertion point would
   be at the end. (Robert Muir)
 
+* LUCENE-5618, LUCENE-5636: SegmentReader referenced unneeded files following 
+  doc-values updates. Now doc-values field updates are written in separate file
+  per field. (Shai Erera, Robert Muir)
+
 Test Framework
 
 * LUCENE-5622: Fail tests if they print over the given limit of bytes to 
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
index d75fceb..8ba8e9e 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
@@ -125,8 +125,7 @@
         }
 
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
-        infos[i].setDocValuesGen(dvGen);
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(atts));
       }
 
       SimpleTextUtil.checkFooter(input);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
index 2053605..6fc195c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
@@ -104,7 +104,7 @@
           attributes.put(LEGACY_NORM_TYPE_KEY, oldNormsType.name());
         }
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, oldValuesType.mapping, oldNormsType.mapping, Collections.unmodifiableMap(attributes));
+          omitNorms, storePayloads, indexOptions, oldValuesType.mapping, oldNormsType.mapping, -1, Collections.unmodifiableMap(attributes));
       }
 
       CodecUtil.checkEOF(input);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
index b9d6934..d38fff0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
@@ -148,12 +148,10 @@
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      // check should be: infos.fieldInfo(fieldNumber) != null, which incorporates negative check
-      // but docvalues updates are currently buggy here (loading extra stuff, etc): LUCENE-5616
-      if (fieldNumber < 0) {
+      if (infos.fieldInfo(fieldNumber) == null) {
         // trickier to validate more: because we re-use for norms, because we use multiple entries
         // for "composite" types like sortedset, etc.
-        throw new CorruptIndexException("Invalid field number: " + fieldNumber + ", input=" + meta);
+        throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
       }
       int fieldType = meta.readByte();
       if (fieldType == NUMBER) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
index c18723d..1919b79 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
@@ -89,7 +89,7 @@
         final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
         final Map<String,String> attributes = input.readStringStringMap();
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, -1, Collections.unmodifiableMap(attributes));
       }
 
       CodecUtil.checkEOF(input);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
index 2523005..54a75d5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
@@ -57,6 +57,7 @@
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.Version;
 import org.apache.lucene.util.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 import org.apache.lucene.util.packed.PackedInts;
@@ -72,13 +73,29 @@
   private final IndexInput data;
   private final int maxDoc;
   private final int version;
+  
+  // We need this for pre-4.9 indexes which recorded multiple fields' DocValues
+  // updates under the same generation, and therefore the passed FieldInfos may
+  // not include all the fields that are encoded in this generation. In that
+  // case, we are more lenient about the fields we read and the passed-in
+  // FieldInfos.
+  @Deprecated
+  private final boolean lenientFieldInfoCheck;
 
   // memory-resident structures
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
   private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
   
   /** expert: instantiates a new reader */
+  @SuppressWarnings("deprecation")
   protected Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+    Version ver;
+    try {
+      ver = Version.parseLeniently(state.segmentInfo.getVersion());
+    } catch (IllegalArgumentException e) {
+      ver = null;
+    }
+    lenientFieldInfoCheck = ver == null || !ver.onOrAfter(Version.LUCENE_4_9);
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     // read in the entries from the metadata file.
     ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
@@ -185,9 +202,7 @@
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      // check should be: infos.fieldInfo(fieldNumber) != null, which incorporates negative check
-      // but docvalues updates are currently buggy here (loading extra stuff, etc): LUCENE-5616
-      if (fieldNumber < 0) {
+      if ((lenientFieldInfoCheck && fieldNumber < 0) || (!lenientFieldInfoCheck && infos.fieldInfo(fieldNumber) == null)) {
         // trickier to validate more: because we re-use for norms, because we use multiple entries
         // for "composite" types like sortedset, etc.
         throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
index 0b24eaf..c65c471 100755
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
@@ -89,8 +89,7 @@
         final long dvGen = input.readLong();
         final Map<String,String> attributes = input.readStringStringMap();
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
-        infos[i].setDocValuesGen(dvGen);
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(attributes));
       }
       
       if (codecVersion >= Lucene46FieldInfosFormat.FORMAT_CHECKSUM) {
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 3729d20..6e7c5ae 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -416,9 +416,9 @@
       pendingUpdates.terms.clear();
       segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
 
-      final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L);
+      final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
       if (infoStream.isEnabled("DWPT")) {
-        infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.segmentInfo.getDocCount() - flushState.delCountOnFlush)) + " deleted docs");
+        infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : flushState.delCountOnFlush) + " deleted docs");
         infoStream.message("DWPT", "new segment has " +
                            (flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
                            (flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " + 
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
index 4aed893..1238fc4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
@@ -47,7 +47,7 @@
 
   private Map<String,String> attributes;
 
-  private long dvGen = -1; // the DocValues generation of this field
+  private long dvGen;
   
   /**
    * Controls how much information is stored in the postings lists.
@@ -121,7 +121,7 @@
    */
   public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, boolean omitNorms, 
       boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, 
-      Map<String,String> attributes) {
+      long dvGen, Map<String,String> attributes) {
     this.name = name;
     this.indexed = indexed;
     this.number = number;
@@ -139,6 +139,7 @@
       this.indexOptions = null;
       this.normType = null;
     }
+    this.dvGen = dvGen;
     this.attributes = attributes;
     assert checkConsistency();
   }
@@ -158,6 +159,10 @@
       // Cannot store payloads unless positions are indexed:
       assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !this.storePayloads;
     }
+    
+    if (dvGen != -1) {
+      assert docValueType != null;
+    }
 
     return true;
   }
@@ -221,8 +226,9 @@
   }
   
   /** Sets the docValues generation of this field. */
-  public void setDocValuesGen(long dvGen) {
+  void setDocValuesGen(long dvGen) {
     this.dvGen = dvGen;
+    assert checkConsistency();
   }
   
   /**
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
index 5d5d326..a803f5d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
@@ -302,7 +302,7 @@
         // before then we'll get the same name and number,
         // else we'll allocate a new one:
         final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues);
-        fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType, null);
+        fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType, -1, null);
         assert !byName.containsKey(fi.name);
         assert globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
         byName.put(fi.name, fi);
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 22f6f7f..e5aeb91 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -2532,7 +2532,7 @@
         }
       }
 
-      SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L);
+      SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
 
       info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
       trackingDir.getCreatedFiles().clear();
@@ -2610,7 +2610,9 @@
     SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
                                           info.info.getUseCompoundFile(), info.info.getCodec(), 
                                           info.info.getDiagnostics());
-    SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(), info.getFieldInfosGen());
+    SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo,
+        info.getDelCount(), info.getDelGen(), info.getFieldInfosGen(),
+        info.getDocValuesGen());
 
     Set<String> segFiles = new HashSet<>();
 
@@ -3733,7 +3735,7 @@
     details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
     details.put("mergeFactor", Integer.toString(merge.segments.size()));
     setDiagnostics(si, SOURCE_MERGE, details);
-    merge.setInfo(new SegmentCommitInfo(si, 0, -1L, -1L));
+    merge.setInfo(new SegmentCommitInfo(si, 0, -1L, -1L, -1L));
 
 //    System.out.println("[" + Thread.currentThread().getName() + "] IW._mergeInit: " + segString(merge.segments) + " into " + si);
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index 20a60f9..553d455 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -29,6 +29,7 @@
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.NumericDocValuesField;
@@ -38,7 +39,6 @@
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MutableBits;
 
 // Used by IndexWriter to hold open SegmentReaders (for
@@ -294,6 +294,170 @@
     
     return true;
   }
+  
+  @SuppressWarnings("synthetic-access")
+  private void handleNumericDVUpdates(FieldInfos infos, Map<String,NumericDocValuesFieldUpdates> updates,
+      Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
+    for (Entry<String,NumericDocValuesFieldUpdates> e : updates.entrySet()) {
+      final String field = e.getKey();
+      final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
+
+      final long nextDocValuesGen = info.getNextDocValuesGen();
+      final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
+      final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
+      final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
+      final FieldInfo fieldInfo = infos.fieldInfo(field);
+      assert fieldInfo != null;
+      fieldInfo.setDocValuesGen(nextDocValuesGen);
+      final FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { fieldInfo });
+      // separately also track which files were created for this gen
+      final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+      final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
+      try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
+        // write the numeric updates to a new gen'd docvalues file
+        fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
+          final NumericDocValues currentValues = reader.getNumericDocValues(field);
+          final Bits docsWithField = reader.getDocsWithField(field);
+          final int maxDoc = reader.maxDoc();
+          final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
+          @Override
+          public Iterator<Number> iterator() {
+            updatesIter.reset();
+            return new Iterator<Number>() {
+
+              int curDoc = -1;
+              int updateDoc = updatesIter.nextDoc();
+              
+              @Override
+              public boolean hasNext() {
+                return curDoc < maxDoc - 1;
+              }
+
+              @Override
+              public Number next() {
+                if (++curDoc >= maxDoc) {
+                  throw new NoSuchElementException("no more documents to return values for");
+                }
+                if (curDoc == updateDoc) { // this document has an updated value
+                  Long value = updatesIter.value(); // either null (unset value) or updated value
+                  updateDoc = updatesIter.nextDoc(); // prepare for next round
+                  return value;
+                } else {
+                  // no update for this document
+                  assert curDoc < updateDoc;
+                  if (currentValues != null && docsWithField.get(curDoc)) {
+                    // only read the current value if the document had a value before
+                    return currentValues.get(curDoc);
+                  } else {
+                    return null;
+                  }
+                }
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException("this iterator does not support removing elements");
+              }
+            };
+          }
+        });
+      }
+      info.advanceDocValuesGen();
+      assert !fieldFiles.containsKey(fieldInfo.number);
+      fieldFiles.put(fieldInfo.number, trackingDir.getCreatedFiles());
+    }
+  }
+
+  @SuppressWarnings("synthetic-access")
+  private void handleBinaryDVUpdates(FieldInfos infos, Map<String,BinaryDocValuesFieldUpdates> updates, 
+      TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
+    for (Entry<String,BinaryDocValuesFieldUpdates> e : updates.entrySet()) {
+      final String field = e.getKey();
+      final BinaryDocValuesFieldUpdates fieldUpdates = e.getValue();
+
+      final long nextDocValuesGen = info.getNextDocValuesGen();
+      final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
+      final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
+      final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
+      final FieldInfo fieldInfo = infos.fieldInfo(field);
+      assert fieldInfo != null;
+      fieldInfo.setDocValuesGen(nextDocValuesGen);
+      final FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { fieldInfo });
+      // separately also track which files were created for this gen
+      final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+      final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
+      try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
+        // write the binary updates to a new gen'd docvalues file
+        fieldsConsumer.addBinaryField(fieldInfo, new Iterable<BytesRef>() {
+          final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
+          final Bits docsWithField = reader.getDocsWithField(field);
+          final int maxDoc = reader.maxDoc();
+          final BinaryDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
+          @Override
+          public Iterator<BytesRef> iterator() {
+            updatesIter.reset();
+            return new Iterator<BytesRef>() {
+              
+              int curDoc = -1;
+              int updateDoc = updatesIter.nextDoc();
+              BytesRef scratch = new BytesRef();
+              
+              @Override
+              public boolean hasNext() {
+                return curDoc < maxDoc - 1;
+              }
+              
+              @Override
+              public BytesRef next() {
+                if (++curDoc >= maxDoc) {
+                  throw new NoSuchElementException("no more documents to return values for");
+                }
+                if (curDoc == updateDoc) { // this document has an updated value
+                  BytesRef value = updatesIter.value(); // either null (unset value) or updated value
+                  updateDoc = updatesIter.nextDoc(); // prepare for next round
+                  return value;
+                } else {
+                  // no update for this document
+                  assert curDoc < updateDoc;
+                  if (currentValues != null && docsWithField.get(curDoc)) {
+                    // only read the current value if the document had a value before
+                    currentValues.get(curDoc, scratch);
+                    return scratch;
+                  } else {
+                    return null;
+                  }
+                }
+              }
+              
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException("this iterator does not support removing elements");
+              }
+            };
+          }
+        });
+      }
+      info.advanceDocValuesGen();
+      assert !fieldFiles.containsKey(fieldInfo.number);
+      fieldFiles.put(fieldInfo.number, trackingDir.getCreatedFiles());
+    }
+  }
+  
+  private Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat, 
+      FieldInfosFormat infosFormat) throws IOException {
+    final long nextFieldInfosGen = info.getNextFieldInfosGen();
+    final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
+    // we write approximately that many bytes (based on Lucene46DVF):
+    // HEADER + FOOTER: 40
+    // 90 bytes per-field (over estimating long name and attributes map)
+    final long estInfosSize = 40 + 90 * fieldInfos.size();
+    final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
+    // separately also track which files were created for this gen
+    final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+    infosFormat.getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
+    info.advanceFieldInfosGen();
+    return trackingDir.getCreatedFiles();
+  }
 
   // Writes field updates (new _X_N updates files) to the directory
   public synchronized void writeFieldUpdates(Directory dir, DocValuesFieldUpdates.Container dvUpdates) throws IOException {
@@ -307,6 +471,8 @@
     // it:
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
     
+    final Map<Integer,Set<String>> newDVFiles = new HashMap<>();
+    Set<String> fieldInfosFiles = null;
     FieldInfos fieldInfos = null;
     boolean success = false;
     try {
@@ -341,147 +507,16 @@
         }
         
         fieldInfos = builder.finish();
-        final long nextFieldInfosGen = info.getNextFieldInfosGen();
-        final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
-        final long estUpdatesSize = dvUpdates.ramBytesPerDoc() * info.info.getDocCount();
-        final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
-        final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
         final DocValuesFormat docValuesFormat = codec.docValuesFormat();
-        final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
-        boolean fieldsConsumerSuccess = false;
-        try {
-//          System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
-          for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
-            final String field = e.getKey();
-            final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
-            final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-            assert fieldInfo != null;
-
-            fieldInfo.setDocValuesGen(nextFieldInfosGen);
-            // write the numeric updates to a new gen'd docvalues file
-            fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
-              final NumericDocValues currentValues = reader.getNumericDocValues(field);
-              final Bits docsWithField = reader.getDocsWithField(field);
-              final int maxDoc = reader.maxDoc();
-              final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
-              @Override
-              public Iterator<Number> iterator() {
-                updatesIter.reset();
-                return new Iterator<Number>() {
-
-                  int curDoc = -1;
-                  int updateDoc = updatesIter.nextDoc();
-                  
-                  @Override
-                  public boolean hasNext() {
-                    return curDoc < maxDoc - 1;
-                  }
-
-                  @Override
-                  public Number next() {
-                    if (++curDoc >= maxDoc) {
-                      throw new NoSuchElementException("no more documents to return values for");
-                    }
-                    if (curDoc == updateDoc) { // this document has an updated value
-                      Long value = updatesIter.value(); // either null (unset value) or updated value
-                      updateDoc = updatesIter.nextDoc(); // prepare for next round
-                      return value;
-                    } else {
-                      // no update for this document
-                      assert curDoc < updateDoc;
-                      if (currentValues != null && docsWithField.get(curDoc)) {
-                        // only read the current value if the document had a value before
-                        return currentValues.get(curDoc);
-                      } else {
-                        return null;
-                      }
-                    }
-                  }
-
-                  @Override
-                  public void remove() {
-                    throw new UnsupportedOperationException("this iterator does not support removing elements");
-                  }
-                };
-              }
-            });
-          }
-
-//        System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
-        for (Entry<String,BinaryDocValuesFieldUpdates> e : dvUpdates.binaryDVUpdates.entrySet()) {
-          final String field = e.getKey();
-          final BinaryDocValuesFieldUpdates dvFieldUpdates = e.getValue();
-          final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-          assert fieldInfo != null;
-
-//          System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " f=" + dvFieldUpdates + ", updates=" + dvFieldUpdates);
-
-          fieldInfo.setDocValuesGen(nextFieldInfosGen);
-          // write the numeric updates to a new gen'd docvalues file
-          fieldsConsumer.addBinaryField(fieldInfo, new Iterable<BytesRef>() {
-            final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
-            final Bits docsWithField = reader.getDocsWithField(field);
-            final int maxDoc = reader.maxDoc();
-            final BinaryDocValuesFieldUpdates.Iterator updatesIter = dvFieldUpdates.iterator();
-            @Override
-            public Iterator<BytesRef> iterator() {
-              updatesIter.reset();
-              return new Iterator<BytesRef>() {
-
-                int curDoc = -1;
-                int updateDoc = updatesIter.nextDoc();
-                BytesRef scratch = new BytesRef();
-                
-                @Override
-                public boolean hasNext() {
-                  return curDoc < maxDoc - 1;
-                }
-
-                @Override
-                public BytesRef next() {
-                  if (++curDoc >= maxDoc) {
-                    throw new NoSuchElementException("no more documents to return values for");
-                  }
-                  if (curDoc == updateDoc) { // this document has an updated value
-                    BytesRef value = updatesIter.value(); // either null (unset value) or updated value
-                    updateDoc = updatesIter.nextDoc(); // prepare for next round
-                    return value;
-                  } else {
-                    // no update for this document
-                    assert curDoc < updateDoc;
-                    if (currentValues != null && docsWithField.get(curDoc)) {
-                      // only read the current value if the document had a value before
-                      currentValues.get(curDoc, scratch);
-                      return scratch;
-                    } else {
-                      return null;
-                    }
-                  }
-                }
-
-                @Override
-                public void remove() {
-                  throw new UnsupportedOperationException("this iterator does not support removing elements");
-                }
-              };
-            }
-          });
-        }
         
-        // we write approximately that many bytes (based on Lucene46DVF):
-        // HEADER + FOOTER: 40
-        // 90 bytes per-field (over estimating long name and attributes map)
-        final long estInfosSize = 40 + 90 * fieldInfos.size();
-        final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
-        codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
-        fieldsConsumerSuccess = true;
-        } finally {
-          if (fieldsConsumerSuccess) {
-            fieldsConsumer.close();
-          } else {
-            IOUtils.closeWhileHandlingException(fieldsConsumer);
-          }
-        }
+//          System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
+        handleNumericDVUpdates(fieldInfos, dvUpdates.numericDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
+        
+//        System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
+        handleBinaryDVUpdates(fieldInfos, dvUpdates.binaryDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
+
+//        System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: write fieldInfos; seg=" + info);
+        fieldInfosFiles = writeFieldInfosGen(fieldInfos, trackingDir, docValuesFormat, codec.fieldInfosFormat());
       } finally {
         if (reader != this.reader) {
 //          System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
@@ -492,9 +527,10 @@
       success = true;
     } finally {
       if (!success) {
-        // Advance only the nextWriteFieldInfosGen so that a 2nd
-        // attempt to write will write to a new file
+        // Advance only the nextWriteFieldInfosGen and nextWriteDocValuesGen, so
+        // that a 2nd attempt to write will write to a new file
         info.advanceNextWriteFieldInfosGen();
+        info.advanceNextWriteDocValuesGen();
         
         // Delete any partially created file(s):
         for (String fileName : trackingDir.getCreatedFiles()) {
@@ -507,7 +543,6 @@
       }
     }
     
-    info.advanceFieldInfosGen();
     // copy all the updates to mergingUpdates, so they can later be applied to the merged segment
     if (isMerging) {
       for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
@@ -528,22 +563,21 @@
       }
     }
     
-    // create a new map, keeping only the gens that are in use
-    Map<Long,Set<String>> genUpdatesFiles = info.getUpdatesFiles();
-    Map<Long,Set<String>> newGenUpdatesFiles = new HashMap<>();
-    final long fieldInfosGen = info.getFieldInfosGen();
-    for (FieldInfo fi : fieldInfos) {
-      long dvGen = fi.getDocValuesGen();
-      if (dvGen != -1 && !newGenUpdatesFiles.containsKey(dvGen)) {
-        if (dvGen == fieldInfosGen) {
-          newGenUpdatesFiles.put(fieldInfosGen, trackingDir.getCreatedFiles());
-        } else {
-          newGenUpdatesFiles.put(dvGen, genUpdatesFiles.get(dvGen));
-        }
+    // writing field updates succeeded
+    assert fieldInfosFiles != null;
+    info.setFieldInfosFiles(fieldInfosFiles);
+    
+    // update the doc-values updates files. the files map each field to its set
+    // of files, hence we copy from the existing map all fields w/ updates that
+    // were not updated in this session, and add new mappings for fields that
+    // were updated now.
+    assert !newDVFiles.isEmpty();
+    for (Entry<Integer,Set<String>> e : info.getDocValuesUpdatesFiles().entrySet()) {
+      if (!newDVFiles.containsKey(e.getKey())) {
+        newDVFiles.put(e.getKey(), e.getValue());
       }
     }
-    
-    info.setGenUpdatesFiles(newGenUpdatesFiles);
+    info.setDocValuesUpdatesFiles(newDVFiles);
     
     // wrote new files, should checkpoint()
     writer.checkpoint();
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
index 8b20e9d..772eab4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java
@@ -51,11 +51,27 @@
   // Generation number of the FieldInfos (-1 if there are no updates)
   private long fieldInfosGen;
   
-  // Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to
+  // Normally 1+fieldInfosGen, unless an exception was hit on last attempt to
   // write
   private long nextWriteFieldInfosGen;
+  
+  // Generation number of the DocValues (-1 if there are no updates)
+  private long docValuesGen;
+  
+  // Normally 1+dvGen, unless an exception was hit on last attempt to
+  // write
+  private long nextWriteDocValuesGen;
 
+  // Track the per-field DocValues update files
+  private final Map<Integer,Set<String>> dvUpdatesFiles = new HashMap<>();
+  
+  // TODO should we add .files() to FieldInfosFormat, like we have on
+  // LiveDocsFormat?
+  // track the fieldInfos update files
+  private final Set<String> fieldInfosFiles = new HashSet<>();
+  
   // Track the per-generation updates files
+  @Deprecated
   private final Map<Long,Set<String>> genUpdatesFiles = new HashMap<>();
   
   private volatile long sizeInBytes = -1;
@@ -71,36 +87,53 @@
    *          deletion generation number (used to name deletion files)
    * @param fieldInfosGen
    *          FieldInfos generation number (used to name field-infos files)
-   **/
-  public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) {
+   * @param docValuesGen
+   *          DocValues generation number (used to name doc-values updates files)
+   */
+  public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen, long docValuesGen) {
     this.info = info;
     this.delCount = delCount;
     this.delGen = delGen;
-    if (delGen == -1) {
-      nextWriteDelGen = 1;
-    } else {
-      nextWriteDelGen = delGen+1;
-    }
-    
+    this.nextWriteDelGen = delGen == -1 ? 1 : delGen + 1;
     this.fieldInfosGen = fieldInfosGen;
-    if (fieldInfosGen == -1) {
-      nextWriteFieldInfosGen = 1;
-    } else {
-      nextWriteFieldInfosGen = fieldInfosGen + 1;
-    }
+    this.nextWriteFieldInfosGen = fieldInfosGen == -1 ? 1 : fieldInfosGen + 1;
+    this.docValuesGen = docValuesGen;
+    this.nextWriteDocValuesGen = docValuesGen == -1 ? 1 : docValuesGen + 1;
   }
 
-  /** Returns the per generation updates files. */
-  public Map<Long,Set<String>> getUpdatesFiles() {
-    return Collections.unmodifiableMap(genUpdatesFiles);
-  }
-  
-  /** Sets the updates file names per generation. Does not deep clone the map. */
+  /**
+   * Sets the updates file names per generation. Does not deep clone the map.
+   * 
+   * @deprecated required to support 4.6-4.8 indexes.
+   */
+  @Deprecated
   public void setGenUpdatesFiles(Map<Long,Set<String>> genUpdatesFiles) {
     this.genUpdatesFiles.clear();
     this.genUpdatesFiles.putAll(genUpdatesFiles);
   }
   
+  /** Returns the per-field DocValues updates files. */
+  public Map<Integer,Set<String>> getDocValuesUpdatesFiles() {
+    return Collections.unmodifiableMap(dvUpdatesFiles);
+  }
+  
+  /** Sets the DocValues updates file names, per field number. Does not deep clone the map. */
+  public void setDocValuesUpdatesFiles(Map<Integer,Set<String>> dvUpdatesFiles) {
+    this.dvUpdatesFiles.clear();
+    this.dvUpdatesFiles.putAll(dvUpdatesFiles);
+  }
+  
+  /** Returns the FieldInfos file names. */
+  public Set<String> getFieldInfosFiles() {
+    return Collections.unmodifiableSet(fieldInfosFiles);
+  }
+  
+  /** Sets the FieldInfos file names. */
+  public void setFieldInfosFiles(Set<String> fieldInfosFiles) {
+    this.fieldInfosFiles.clear();
+    this.fieldInfosFiles.addAll(fieldInfosFiles);
+  }
+
   /** Called when we succeed in writing deletes */
   void advanceDelGen() {
     delGen = nextWriteDelGen;
@@ -129,6 +162,21 @@
   void advanceNextWriteFieldInfosGen() {
     nextWriteFieldInfosGen++;
   }
+  
+  /** Called when we succeed in writing a new DocValues generation. */
+  void advanceDocValuesGen() {
+    docValuesGen = nextWriteDocValuesGen;
+    nextWriteDocValuesGen = docValuesGen + 1;
+    sizeInBytes = -1;
+  }
+  
+  /**
+   * Called if there was an exception while writing a new generation of
+   * DocValues, so that we don't try to write to the same file more than once.
+   */
+  void advanceNextWriteDocValuesGen() {
+    nextWriteDocValuesGen++;
+  }
 
   /** Returns total size in bytes of all files for this
    *  segment. */
@@ -155,11 +203,20 @@
     // Must separately add any live docs files:
     info.getCodec().liveDocsFormat().files(this, files);
 
-    // Must separately add any field updates files
+    // Must separately add any per-gen updates files. This can go away when we
+    // get rid of genUpdatesFiles (6.0)
     for (Set<String> updateFiles : genUpdatesFiles.values()) {
       files.addAll(updateFiles);
     }
     
+    // must separately add any field updates files
+    for (Set<String> updatefiles : dvUpdatesFiles.values()) {
+      files.addAll(updatefiles);
+    }
+    
+    // must separately add fieldInfos files
+    files.addAll(fieldInfosFiles);
+    
     return files;
   }
 
@@ -200,6 +257,19 @@
     return fieldInfosGen;
   }
   
+  /** Returns the next available generation number of the DocValues files. */
+  public long getNextDocValuesGen() {
+    return nextWriteDocValuesGen;
+  }
+  
+  /**
+   * Returns the generation number of the DocValues file or -1 if there are no
+   * doc-values updates yet.
+   */
+  public long getDocValuesGen() {
+    return docValuesGen;
+  }
+  
   /**
    * Returns the next available generation number
    * of the live docs file.
@@ -239,6 +309,9 @@
     if (fieldInfosGen != -1) {
       s += ":fieldInfosGen=" + fieldInfosGen;
     }
+    if (docValuesGen != -1) {
+      s += ":dvGen=" + docValuesGen;
+    }
     return s;
   }
 
@@ -249,19 +322,27 @@
 
   @Override
   public SegmentCommitInfo clone() {
-    SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen);
+    SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, docValuesGen);
     // Not clear that we need to carry over nextWriteDelGen
     // (i.e. do we ever clone after a failed write and
     // before the next successful write?), but just do it to
     // be safe:
     other.nextWriteDelGen = nextWriteDelGen;
     other.nextWriteFieldInfosGen = nextWriteFieldInfosGen;
+    other.nextWriteDocValuesGen = nextWriteDocValuesGen;
     
     // deep clone
     for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
       other.genUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue()));
     }
     
+    // deep clone
+    for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
+      other.dvUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue()));
+    }
+    
+    other.fieldInfosFiles.addAll(fieldInfosFiles);
+    
     return other;
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
index edd1f56..ffb4fba 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
@@ -38,7 +38,7 @@
   private final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<>();
 
   private RefCount<DocValuesProducer> newDocValuesProducer(SegmentCommitInfo si, IOContext context, Directory dir,
-      DocValuesFormat dvFormat, final Long gen, List<FieldInfo> infos) throws IOException {
+      DocValuesFormat dvFormat, final Long gen, FieldInfos infos) throws IOException {
     Directory dvDir = dir;
     String segmentSuffix = "";
     if (gen.longValue() != -1) {
@@ -47,7 +47,7 @@
     }
 
     // set SegmentReadState to list only the fields that are relevant to that gen
-    SegmentReadState srs = new SegmentReadState(dvDir, si.info, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])), context, segmentSuffix);
+    SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, context, segmentSuffix);
     return new RefCount<DocValuesProducer>(dvFormat.fieldsProducer(srs)) {
       @SuppressWarnings("synthetic-access")
       @Override
@@ -62,7 +62,7 @@
 
   /** Returns the {@link DocValuesProducer} for the given generation. */
   synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentCommitInfo si, IOContext context, Directory dir, 
-      DocValuesFormat dvFormat, List<FieldInfo> infos) throws IOException {
+      DocValuesFormat dvFormat, FieldInfos infos) throws IOException {
     RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
     if (dvp == null) {
       dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos);
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 353c91b..38c64d2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -33,6 +33,7 @@
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.store.ChecksumIndexInput;
@@ -44,68 +45,79 @@
 import org.apache.lucene.util.IOUtils;
 
 /**
- * A collection of segmentInfo objects with methods for operating on
- * those segments in relation to the file system.
+ * A collection of segmentInfo objects with methods for operating on those
+ * segments in relation to the file system.
  * <p>
  * The active segments in the index are stored in the segment info file,
- * <tt>segments_N</tt>. There may be one or more <tt>segments_N</tt> files in the
- * index; however, the one with the largest generation is the active one (when
- * older segments_N files are present it's because they temporarily cannot be
- * deleted, or, a writer is in the process of committing, or a custom 
- * {@link org.apache.lucene.index.IndexDeletionPolicy IndexDeletionPolicy}
- * is in use). This file lists each segment by name and has details about the
- * codec and generation of deletes.
+ * <tt>segments_N</tt>. There may be one or more <tt>segments_N</tt> files in
+ * the index; however, the one with the largest generation is the active one
+ * (when older segments_N files are present it's because they temporarily cannot
+ * be deleted, or, a writer is in the process of committing, or a custom
+ * {@link org.apache.lucene.index.IndexDeletionPolicy IndexDeletionPolicy} is in
+ * use). This file lists each segment by name and has details about the codec
+ * and generation of deletes.
  * </p>
- * <p>There is also a file <tt>segments.gen</tt>. This file contains
- * the current generation (the <tt>_N</tt> in <tt>segments_N</tt>) of the index.
- * This is used only as a fallback in case the current generation cannot be
- * accurately determined by directory listing alone (as is the case for some NFS
- * clients with time-based directory cache expiration). This file simply contains
- * an {@link DataOutput#writeInt Int32} version header 
- * ({@link #FORMAT_SEGMENTS_GEN_CURRENT}), followed by the
- * generation recorded as {@link DataOutput#writeLong Int64}, written twice.</p>
+ * <p>
+ * There is also a file <tt>segments.gen</tt>. This file contains the current
+ * generation (the <tt>_N</tt> in <tt>segments_N</tt>) of the index. This is
+ * used only as a fallback in case the current generation cannot be accurately
+ * determined by directory listing alone (as is the case for some NFS clients
+ * with time-based directory cache expiration). This file simply contains an
+ * {@link DataOutput#writeInt Int32} version header (
+ * {@link #FORMAT_SEGMENTS_GEN_CURRENT}), followed by the generation recorded as
+ * {@link DataOutput#writeLong Int64}, written twice.
+ * </p>
  * <p>
  * Files:
  * <ul>
- *   <li><tt>segments.gen</tt>: GenHeader, Generation, Generation, Footer
- *   <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount,
- *    &lt;SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles&gt;<sup>SegCount</sup>, 
- *    CommitUserData, Footer
+ * <li><tt>segments.gen</tt>: GenHeader, Generation, Generation, Footer
+ * <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount, &lt;SegName,
+ * SegCodec, DelGen, DeletionCount, FieldInfosGen, DocValuesGen,
+ * UpdatesFiles&gt;<sup>SegCount</sup>, CommitUserData, Footer
  * </ul>
  * </p>
  * Data types:
  * <p>
  * <ul>
- *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *   <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt; {@link DataOutput#writeInt Int32}</li>
- *   <li>Generation, Version, DelGen, Checksum, FieldInfosGen --&gt; {@link DataOutput#writeLong Int64}</li>
- *   <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
- *   <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
- *   <li>UpdatesFiles --&gt; {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
+ * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ * <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt;
+ * {@link DataOutput#writeInt Int32}</li>
+ * <li>Generation, Version, DelGen, Checksum, FieldInfosGen, DocValuesGen --&gt;
+ * {@link DataOutput#writeLong Int64}</li>
+ * <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
+ * <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap
+ * Map&lt;String,String&gt;}</li>
+ * <li>UpdatesFiles --&gt; Map&lt;{@link DataOutput#writeInt Int32},
+ * {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}&gt;</li>
+ * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * </p>
  * Field Descriptions:
  * <p>
  * <ul>
- *   <li>Version counts how often the index has been changed by adding or deleting
- *       documents.</li>
- *   <li>NameCounter is used to generate names for new segment files.</li>
- *   <li>SegName is the name of the segment, and is used as the file name prefix for
- *       all of the files that compose the segment's index.</li>
- *   <li>DelGen is the generation count of the deletes file. If this is -1,
- *       there are no deletes. Anything above zero means there are deletes 
- *       stored by {@link LiveDocsFormat}.</li>
- *   <li>DeletionCount records the number of deleted documents in this segment.</li>
- *   <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
- *       this segment.</li>
- *   <li>CommitUserData stores an optional user-supplied opaque
- *       Map&lt;String,String&gt; that was passed to 
- *       {@link IndexWriter#setCommitData(java.util.Map)}.</li>
- *   <li>FieldInfosGen is the generation count of the fieldInfos file. If this is -1,
- *       there are no updates to the fieldInfos in that segment. Anything above zero 
- *       means there are updates to fieldInfos stored by {@link FieldInfosFormat}.</li>
- *   <li>UpdatesFiles stores the list of files that were updated in that segment.</li>
+ * <li>Version counts how often the index has been changed by adding or deleting
+ * documents.</li>
+ * <li>NameCounter is used to generate names for new segment files.</li>
+ * <li>SegName is the name of the segment, and is used as the file name prefix
+ * for all of the files that compose the segment's index.</li>
+ * <li>DelGen is the generation count of the deletes file. If this is -1, there
+ * are no deletes. Anything above zero means there are deletes stored by
+ * {@link LiveDocsFormat}.</li>
+ * <li>DeletionCount records the number of deleted documents in this segment.</li>
+ * <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
+ * this segment.</li>
+ * <li>CommitUserData stores an optional user-supplied opaque
+ * Map&lt;String,String&gt; that was passed to
+ * {@link IndexWriter#setCommitData(java.util.Map)}.</li>
+ * <li>FieldInfosGen is the generation count of the fieldInfos file. If this is
+ * -1, there are no updates to the fieldInfos in that segment. Anything above
+ * zero means there are updates to fieldInfos stored by {@link FieldInfosFormat}
+ * .</li>
+ * <li>DocValuesGen is the generation count of the updatable DocValues. If this
+ * is -1, there are no updates to DocValues in that segment. Anything above zero
+ * means there are updates to DocValues stored by {@link DocValuesFormat}.</li>
+ * <li>UpdatesFiles stores the set of files that were updated in that segment
+ * per field.</li>
  * </ul>
  * </p>
  * 
@@ -121,6 +133,9 @@
   
   /** The file format version for the segments_N codec header, since 4.8+ */
   public static final int VERSION_48 = 2;
+  
+  /** The file format version for the segments_N codec header, since 4.9+ */
+  public static final int VERSION_49 = 3;
 
   // Used for the segments.gen file only!
   // Whenever you add a new format, make it 1 smaller (negative version logic)!
@@ -330,7 +345,7 @@
         throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
       }
       // 4.0+
-      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_48);
+      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_49);
       version = input.readLong();
       counter = input.readInt();
       int numSegments = input.readInt();
@@ -352,19 +367,45 @@
         if (format >= VERSION_46) {
           fieldInfosGen = input.readLong();
         }
-        SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen);
+        long dvGen = -1;
+        if (format >= VERSION_49) {
+          dvGen = input.readLong();
+        } else {
+          dvGen = fieldInfosGen;
+        }
+        SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
         if (format >= VERSION_46) {
-          int numGensUpdatesFiles = input.readInt();
-          final Map<Long,Set<String>> genUpdatesFiles;
-          if (numGensUpdatesFiles == 0) {
-            genUpdatesFiles = Collections.emptyMap();
-          } else {
-            genUpdatesFiles = new HashMap<>(numGensUpdatesFiles);
-            for (int i = 0; i < numGensUpdatesFiles; i++) {
-              genUpdatesFiles.put(input.readLong(), input.readStringSet());
+          if (format < VERSION_49) {
+            // Recorded per-generation files, which were buggy (see
+            // LUCENE-5636). We need to read and keep them so we continue to
+            // reference those files. Unfortunately it means that the files will
+            // be referenced even if the fields are updated again, until the
+            // segment is merged.
+            final int numGensUpdatesFiles = input.readInt();
+            final Map<Long,Set<String>> genUpdatesFiles;
+            if (numGensUpdatesFiles == 0) {
+              genUpdatesFiles = Collections.emptyMap();
+            } else {
+              genUpdatesFiles = new HashMap<>(numGensUpdatesFiles);
+              for (int i = 0; i < numGensUpdatesFiles; i++) {
+                genUpdatesFiles.put(input.readLong(), input.readStringSet());
+              }
             }
+            siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
+          } else {
+            siPerCommit.setFieldInfosFiles(input.readStringSet());
+            final Map<Integer,Set<String>> dvUpdateFiles;
+            final int numDVFields = input.readInt();
+            if (numDVFields == 0) {
+              dvUpdateFiles = Collections.emptyMap();
+            } else {
+              dvUpdateFiles = new HashMap<>(numDVFields);
+              for (int i = 0; i < numDVFields; i++) {
+                dvUpdateFiles.put(input.readInt(), input.readStringSet());
+              }
+            }
+            siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
           }
-          siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
         }
         add(siPerCommit);
       }
@@ -429,7 +470,7 @@
 
     try {
       segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
-      CodecUtil.writeHeader(segnOutput, "segments", VERSION_48);
+      CodecUtil.writeHeader(segnOutput, "segments", VERSION_49);
       segnOutput.writeLong(version); 
       segnOutput.writeInt(counter); // write counter
       segnOutput.writeInt(size()); // write infos
@@ -444,10 +485,12 @@
         }
         segnOutput.writeInt(delCount);
         segnOutput.writeLong(siPerCommit.getFieldInfosGen());
-        final Map<Long,Set<String>> genUpdatesFiles = siPerCommit.getUpdatesFiles();
-        segnOutput.writeInt(genUpdatesFiles.size());
-        for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
-          segnOutput.writeLong(e.getKey());
+        segnOutput.writeLong(siPerCommit.getDocValuesGen());
+        segnOutput.writeStringSet(siPerCommit.getFieldInfosFiles());
+        final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
+        segnOutput.writeInt(dvUpdatesFiles.size());
+        for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
+          segnOutput.writeInt(e.getKey());
           segnOutput.writeStringSet(e.getValue());
         }
         assert si.dir == directory;
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
index 03f4043..985b0a1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
@@ -23,7 +23,6 @@
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.List;
-import java.util.Map.Entry;
 import java.util.Map;
 import java.util.Set;
 
@@ -41,6 +40,7 @@
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.Version;
 
 /**
  * IndexReader implementation over a single segment. 
@@ -112,7 +112,7 @@
         liveDocs = null;
       }
       numDocs = si.info.getDocCount() - si.getDelCount();
-      
+
       if (fieldInfos.hasDocValues()) {
         initDocValuesProducers(codec);
       }
@@ -175,24 +175,88 @@
   }
 
   // initialize the per-field DocValuesProducer
+  @SuppressWarnings("deprecation")
   private void initDocValuesProducers(Codec codec) throws IOException {
     final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
     final DocValuesFormat dvFormat = codec.docValuesFormat();
-    final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
-    
-//      System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gens=" + genInfos.keySet());
-    
-    // TODO: can we avoid iterating over fieldinfos several times and creating maps of all this stuff if dv updates do not exist?
-    
-    for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
-      Long gen = e.getKey();
-      List<FieldInfo> infos = e.getValue();
-      DocValuesProducer dvp = segDocValues.getDocValuesProducer(gen, si, IOContext.READ, dir, dvFormat, infos);
-      dvGens.add(gen);
-      for (FieldInfo fi : infos) {
+
+    if (!si.hasFieldUpdates()) {
+      // simple case, no DocValues updates
+      final DocValuesProducer dvp = segDocValues.getDocValuesProducer(-1L, si, IOContext.READ, dir, dvFormat, fieldInfos);
+      dvGens.add(-1L);
+      dvProducers.add(dvp);
+      for (FieldInfo fi : fieldInfos) {
+        if (!fi.hasDocValues()) continue;
+        assert fi.getDocValuesGen() == -1;
         dvProducersByField.put(fi.name, dvp);
       }
-      dvProducers.add(dvp);
+      return;
+    }
+
+    Version ver;
+    try {
+      ver = Version.parseLeniently(si.info.getVersion());
+    } catch (IllegalArgumentException e) {
+      // happened in TestBackwardsCompatibility on a 4.0.0.2 index (no matching
+      // Version constant), anyway it's a pre-4.9 index.
+      ver = null;
+    }
+    if (ver != null && ver.onOrAfter(Version.LUCENE_4_9)) {
+      DocValuesProducer baseProducer = null;
+      for (FieldInfo fi : fieldInfos) {
+        if (!fi.hasDocValues()) continue;
+        long docValuesGen = fi.getDocValuesGen();
+        if (docValuesGen == -1) {
+          if (baseProducer == null) {
+//        System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
+            // the base producer gets all the fields, so the Codec can validate properly
+            baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
+            dvGens.add(docValuesGen);
+            dvProducers.add(baseProducer);
+          }
+//        System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
+          dvProducersByField.put(fi.name, baseProducer);
+        } else {
+          assert !dvGens.contains(docValuesGen);
+//        System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
+          final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(new FieldInfo[] { fi }));
+          dvGens.add(docValuesGen);
+          dvProducers.add(dvp);
+          dvProducersByField.put(fi.name, dvp);
+        }
+      }
+    } else {
+      // For pre-4.9 indexes, especially with doc-values updates, multiple
+      // FieldInfos could belong to the same dvGen. Therefore need to make sure
+      // we initialize each DocValuesProducer once per gen.
+      Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
+      for (FieldInfo fi : fieldInfos) {
+        if (!fi.hasDocValues()) continue;
+        List<FieldInfo> genFieldInfos = genInfos.get(fi.getDocValuesGen());
+        if (genFieldInfos == null) {
+          genFieldInfos = new ArrayList<>();
+          genInfos.put(fi.getDocValuesGen(), genFieldInfos);
+        }
+        genFieldInfos.add(fi);
+      }
+      
+      for (Map.Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
+        long docValuesGen = e.getKey();
+        List<FieldInfo> infos = e.getValue();
+        final DocValuesProducer dvp;
+        if (docValuesGen == -1) {
+          // we need to send all FieldInfos to gen=-1, but later we need to
+          // record the DVP only for the "true" gen=-1 fields (not updated)
+          dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
+        } else {
+          dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])));
+        }
+        dvGens.add(docValuesGen);
+        dvProducers.add(dvp);
+        for (FieldInfo fi : infos) {
+          dvProducersByField.put(fi.name, dvp);
+        }
+      }
     }
   }
   
@@ -229,24 +293,6 @@
     }
   }
   
-  // returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
-  private Map<Long,List<FieldInfo>> getGenInfos() {
-    final Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
-    for (FieldInfo fi : fieldInfos) {
-      if (fi.getDocValuesType() == null) {
-        continue;
-      }
-      long gen = fi.getDocValuesGen();
-      List<FieldInfo> infos = genInfos.get(gen);
-      if (infos == null) {
-        infos = new ArrayList<>();
-        genInfos.put(gen, infos);
-      }
-      infos.add(fi);
-    }
-    return genInfos;
-  }
-
   @Override
   public Bits getLiveDocs() {
     ensureOpen();
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
index aad0540..f6de712 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
@@ -341,6 +342,11 @@
     @Override
     public Scorer scorer(AtomicReaderContext context, Bits acceptDocs)
         throws IOException {
+      // initially the user provided value,
+      // but if minNrShouldMatch == optional.size(),
+      // we will optimize and move these to required, making this 0
+      int minShouldMatch = minNrShouldMatch;
+
       List<Scorer> required = new ArrayList<>();
       List<Scorer> prohibited = new ArrayList<>();
       List<Scorer> optional = new ArrayList<>();
@@ -360,34 +366,69 @@
           optional.add(subScorer);
         }
       }
-
-      if (required.size() == 0 && optional.size() == 0) {
+      
+      // scorer simplifications:
+      
+      if (optional.size() == minShouldMatch) {
+        // any optional clauses are in fact required
+        required.addAll(optional);
+        optional.clear();
+        minShouldMatch = 0;
+      }
+      
+      if (required.isEmpty() && optional.isEmpty()) {
         // no required and optional clauses.
         return null;
-      } else if (optional.size() < minNrShouldMatch) {
+      } else if (optional.size() < minShouldMatch) {
         // either >1 req scorer, or there are 0 req scorers and at least 1
         // optional scorer. Therefore if there are not enough optional scorers
         // no documents will be matched by the query
         return null;
       }
       
-      // simple conjunction
-      if (optional.size() == 0 && prohibited.size() == 0) {
-        float coord = disableCoord ? 1.0f : coord(required.size(), maxCoord);
-        return new ConjunctionScorer(this, required.toArray(new Scorer[required.size()]), coord);
+      // three cases: conjunction, disjunction, or mix
+      
+      // pure conjunction
+      if (optional.isEmpty()) {
+        return excl(req(required, disableCoord), prohibited);
       }
       
-      // simple disjunction
-      if (required.size() == 0 && prohibited.size() == 0 && minNrShouldMatch <= 1 && optional.size() > 1) {
-        float coord[] = new float[optional.size()+1];
-        for (int i = 0; i < coord.length; i++) {
-          coord[i] = disableCoord ? 1.0f : coord(i, maxCoord);
+      // pure disjunction
+      if (required.isEmpty()) {
+        return excl(opt(optional, minShouldMatch, disableCoord), prohibited);
+      }
+      
+      // conjunction-disjunction mix:
+      // we create the required and optional pieces with coord disabled, and then
+      // combine the two: if minNrShouldMatch > 0, then its a conjunction: because the
+      // optional side must match. otherwise its required + optional, factoring the
+      // number of optional terms into the coord calculation
+      
+      Scorer req = excl(req(required, true), prohibited);
+      Scorer opt = opt(optional, minShouldMatch, true);
+
+      // TODO: clean this up: its horrible
+      if (disableCoord) {
+        if (minShouldMatch > 0) {
+          return new ConjunctionScorer(this, new Scorer[] { req, opt }, 1F);
+        } else {
+          return new ReqOptSumScorer(req, opt);          
         }
-        return new DisjunctionSumScorer(this, optional.toArray(new Scorer[optional.size()]), coord);
+      } else if (optional.size() == 1) {
+        if (minShouldMatch > 0) {
+          return new ConjunctionScorer(this, new Scorer[] { req, opt }, coord(required.size()+1, maxCoord));
+        } else {
+          float coordReq = coord(required.size(), maxCoord);
+          float coordBoth = coord(required.size() + 1, maxCoord);
+          return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
+        }
+      } else {
+        if (minShouldMatch > 0) {
+          return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords(), req, required.size(), opt);
+        } else {
+          return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, required.size(), coords()); 
+        }
       }
-      
-      // Return a BooleanScorer2
-      return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
     }
     
     @Override
@@ -396,17 +437,89 @@
         // BS2 (in-order) will be used by scorer()
         return false;
       }
+      int optionalCount = 0;
       for (BooleanClause c : clauses) {
         if (c.isRequired()) {
           // BS2 (in-order) will be used by scorer()
           return false;
+        } else if (!c.isProhibited()) {
+          optionalCount++;
         }
       }
       
+      if (optionalCount == minNrShouldMatch) {
+        return false; // BS2 (in-order) will be used, as this means conjunction
+      }
+      
       // scorer() will return an out-of-order scorer if requested.
       return true;
     }
     
+    private Scorer req(List<Scorer> required, boolean disableCoord) {
+      if (required.size() == 1) {
+        Scorer req = required.get(0);
+        if (!disableCoord && maxCoord > 1) {
+          return new BooleanTopLevelScorers.BoostedScorer(req, coord(1, maxCoord));
+        } else {
+          return req;
+        }
+      } else {
+        return new ConjunctionScorer(this, 
+                                     required.toArray(new Scorer[required.size()]),
+                                     disableCoord ? 1.0F : coord(required.size(), maxCoord));
+      }
+    }
+    
+    private Scorer excl(Scorer main, List<Scorer> prohibited) throws IOException {
+      if (prohibited.isEmpty()) {
+        return main;
+      } else if (prohibited.size() == 1) {
+        return new ReqExclScorer(main, prohibited.get(0));
+      } else {
+        float coords[] = new float[prohibited.size()+1];
+        Arrays.fill(coords, 1F);
+        // TODO: don't score here.
+        return new ReqExclScorer(main, 
+                                 new DisjunctionSumScorer(this, 
+                                                          prohibited.toArray(new Scorer[prohibited.size()]), 
+                                                          coords));
+      }
+    }
+    
+    private Scorer opt(List<Scorer> optional, int minShouldMatch, boolean disableCoord) throws IOException {
+      if (optional.size() == 1) {
+        Scorer opt = optional.get(0);
+        if (!disableCoord && maxCoord > 1) {
+          return new BooleanTopLevelScorers.BoostedScorer(opt, coord(1, maxCoord));
+        } else {
+          return opt;
+        }
+      } else {
+        float coords[];
+        if (disableCoord) {
+          coords = new float[optional.size()+1];
+          Arrays.fill(coords, 1F);
+        } else {
+          coords = coords();
+        }
+        if (minShouldMatch > 1) {
+          return new MinShouldMatchSumScorer(this, optional, minShouldMatch, coords);
+        } else {
+          return new DisjunctionSumScorer(this, 
+                                          optional.toArray(new Scorer[optional.size()]), 
+                                          coords);
+        }
+      }
+    }
+    
+    private float[] coords() {
+      float[] coords = new float[maxCoord+1];
+      coords[0] = 0F;
+      for (int i = 1; i < coords.length; i++) {
+        coords[i] = coord(i, maxCoord);
+      }
+      return coords;
+    }
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
deleted file mode 100644
index 8e50635..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
+++ /dev/null
@@ -1,328 +0,0 @@
-package org.apache.lucene.search;
-
-/*
- * 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.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery.BooleanWeight;
-import org.apache.lucene.search.similarities.Similarity;
-
-/* See the description in BooleanScorer.java, comparing
- * BooleanScorer & BooleanScorer2 */
-
-/** An alternative to BooleanScorer that also allows a minimum number
- * of optional scorers that should match.
- * <br>Implements skipTo(), and has no limitations on the numbers of added scorers.
- * <br>Uses ConjunctionScorer, DisjunctionScorer, ReqOptScorer and ReqExclScorer.
- */
-class BooleanScorer2 extends Scorer {
-  
-  private final List<Scorer> requiredScorers;
-  private final List<Scorer> optionalScorers;
-  private final List<Scorer> prohibitedScorers;
-
-  private class Coordinator {
-    final float coordFactors[];
-
-    Coordinator(int maxCoord, boolean disableCoord) {
-      coordFactors = new float[optionalScorers.size() + requiredScorers.size() + 1];
-      for (int i = 0; i < coordFactors.length; i++) {
-        coordFactors[i] = disableCoord ? 1.0f : ((BooleanWeight)weight).coord(i, maxCoord);
-      }
-    }
-    
-    int nrMatchers; // to be increased by score() of match counting scorers.
-  }
-
-  private final Coordinator coordinator;
-
-  /** The scorer to which all scoring will be delegated,
-   * except for computing and using the coordination factor.
-   */
-  private final Scorer countingSumScorer;
-
-  /** The number of optionalScorers that need to match (if there are any) */
-  private final int minNrShouldMatch;
-
-  private int doc = -1;
-
-  /**
-   * Creates a {@link Scorer} with the given similarity and lists of required,
-   * prohibited and optional scorers. In no required scorers are added, at least
-   * one of the optional scorers will have to match during the search.
-   * 
-   * @param weight
-   *          The BooleanWeight to be used.
-   * @param disableCoord
-   *          If this parameter is true, coordination level matching 
-   *          ({@link Similarity#coord(int, int)}) is not used.
-   * @param minNrShouldMatch
-   *          The minimum number of optional added scorers that should match
-   *          during the search. In case no required scorers are added, at least
-   *          one of the optional scorers will have to match during the search.
-   * @param required
-   *          the list of required scorers.
-   * @param prohibited
-   *          the list of prohibited scorers.
-   * @param optional
-   *          the list of optional scorers.
-   */
-  public BooleanScorer2(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
-      List<Scorer> required, List<Scorer> prohibited, List<Scorer> optional, int maxCoord) throws IOException {
-    super(weight);
-    if (minNrShouldMatch < 0) {
-      throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
-    }
-    this.minNrShouldMatch = minNrShouldMatch;
-
-    optionalScorers = optional;
-    requiredScorers = required;    
-    prohibitedScorers = prohibited;
-    coordinator = new Coordinator(maxCoord, disableCoord);
-    
-    countingSumScorer = makeCountingSumScorer(disableCoord);
-  }
-  
-  /** Count a scorer as a single match. */
-  private class SingleMatchScorer extends Scorer {
-    private Scorer scorer;
-    private int lastScoredDoc = -1;
-    // Save the score of lastScoredDoc, so that we don't compute it more than
-    // once in score().
-    private float lastDocScore = Float.NaN;
-
-    SingleMatchScorer(Scorer scorer) {
-      super(scorer.weight);
-      this.scorer = scorer;
-    }
-
-    @Override
-    public float score() throws IOException {
-      int doc = docID();
-      if (doc >= lastScoredDoc) {
-        if (doc > lastScoredDoc) {
-          lastDocScore = scorer.score();
-          lastScoredDoc = doc;
-        }
-        coordinator.nrMatchers++;
-      }
-      return lastDocScore;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public int docID() {
-      return scorer.docID();
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return scorer.nextDoc();
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return scorer.advance(target);
-    }
-
-    @Override
-    public long cost() {
-      return scorer.cost();
-    }
-  }
-
-  private Scorer countingDisjunctionSumScorer(final List<Scorer> scorers,
-      int minNrShouldMatch) throws IOException {
-    // each scorer from the list counted as a single matcher
-    if (minNrShouldMatch > 1) {
-      return new MinShouldMatchSumScorer(weight, scorers, minNrShouldMatch) {
-        @Override 
-        public float score() throws IOException {
-          coordinator.nrMatchers += super.nrMatchers;
-          return super.score();
-        }
-      };
-    } else {
-      // we pass null for coord[] since we coordinate ourselves and override score()
-      return new DisjunctionSumScorer(weight, scorers.toArray(new Scorer[scorers.size()]), null) {
-        @Override 
-        public float score() throws IOException {
-          coordinator.nrMatchers += super.nrMatchers;
-          return (float) super.score;
-        }
-      };
-    }
-  }
-
-  private Scorer countingConjunctionSumScorer(boolean disableCoord,
-                                              List<Scorer> requiredScorers) throws IOException {
-    // each scorer from the list counted as a single matcher
-    final int requiredNrMatchers = requiredScorers.size();
-    return new ConjunctionScorer(weight, requiredScorers.toArray(new Scorer[requiredScorers.size()])) {
-      private int lastScoredDoc = -1;
-      // Save the score of lastScoredDoc, so that we don't compute it more than
-      // once in score().
-      private float lastDocScore = Float.NaN;
-      @Override public float score() throws IOException {
-        int doc = docID();
-        if (doc >= lastScoredDoc) {
-          if (doc > lastScoredDoc) {
-            lastDocScore = super.score();
-            lastScoredDoc = doc;
-          }
-          coordinator.nrMatchers += requiredNrMatchers;
-        }
-        // All scorers match, so defaultSimilarity super.score() always has 1 as
-        // the coordination factor.
-        // Therefore the sum of the scores of the requiredScorers
-        // is used as score.
-        return lastDocScore;
-      }
-    };
-  }
-
-  private Scorer dualConjunctionSumScorer(boolean disableCoord,
-                                                Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(weight, new Scorer[] { req1, req2 });
-    // All scorers match, so defaultSimilarity always has 1 as
-    // the coordination factor.
-    // Therefore the sum of the scores of two scorers
-    // is used as score.
-  }
-
-  /** Returns the scorer to be used for match counting and score summing.
-   * Uses requiredScorers, optionalScorers and prohibitedScorers.
-   */
-  private Scorer makeCountingSumScorer(boolean disableCoord) throws IOException { // each scorer counted as a single matcher
-    return (requiredScorers.size() == 0)
-      ? makeCountingSumScorerNoReq(disableCoord)
-      : makeCountingSumScorerSomeReq(disableCoord);
-  }
-
-  private Scorer makeCountingSumScorerNoReq(boolean disableCoord) throws IOException { // No required scorers
-    // minNrShouldMatch optional scorers are required, but at least 1
-    int nrOptRequired = (minNrShouldMatch < 1) ? 1 : minNrShouldMatch;
-    Scorer requiredCountingSumScorer;
-    if (optionalScorers.size() > nrOptRequired)
-      requiredCountingSumScorer = countingDisjunctionSumScorer(optionalScorers, nrOptRequired);
-    else if (optionalScorers.size() == 1)
-      requiredCountingSumScorer = new SingleMatchScorer(optionalScorers.get(0));
-    else {
-      requiredCountingSumScorer = countingConjunctionSumScorer(disableCoord, optionalScorers);
-    }
-    return addProhibitedScorers(requiredCountingSumScorer);
-  }
-
-  private Scorer makeCountingSumScorerSomeReq(boolean disableCoord) throws IOException { // At least one required scorer.
-    if (optionalScorers.size() == minNrShouldMatch) { // all optional scorers also required.
-      ArrayList<Scorer> allReq = new ArrayList<>(requiredScorers);
-      allReq.addAll(optionalScorers);
-      return addProhibitedScorers(countingConjunctionSumScorer(disableCoord, allReq));
-    } else { // optionalScorers.size() > minNrShouldMatch, and at least one required scorer
-      Scorer requiredCountingSumScorer =
-            requiredScorers.size() == 1
-            ? new SingleMatchScorer(requiredScorers.get(0))
-            : countingConjunctionSumScorer(disableCoord, requiredScorers);
-      if (minNrShouldMatch > 0) { // use a required disjunction scorer over the optional scorers
-        return addProhibitedScorers( 
-                      dualConjunctionSumScorer( // non counting
-                              disableCoord,
-                              requiredCountingSumScorer,
-                              countingDisjunctionSumScorer(
-                                      optionalScorers,
-                                      minNrShouldMatch)));
-      } else { // minNrShouldMatch == 0
-        return new ReqOptSumScorer(
-                      addProhibitedScorers(requiredCountingSumScorer),
-                      optionalScorers.size() == 1
-                        ? new SingleMatchScorer(optionalScorers.get(0))
-                        // require 1 in combined, optional scorer.
-                        : countingDisjunctionSumScorer(optionalScorers, 1));
-      }
-    }
-  }
-  
-  /** Returns the scorer to be used for match counting and score summing.
-   * Uses the given required scorer and the prohibitedScorers.
-   * @param requiredCountingSumScorer A required scorer already built.
-   */
-  private Scorer addProhibitedScorers(Scorer requiredCountingSumScorer) throws IOException
-  {
-    return (prohibitedScorers.size() == 0)
-          ? requiredCountingSumScorer // no prohibited
-          : new ReqExclScorer(requiredCountingSumScorer,
-                              ((prohibitedScorers.size() == 1)
-                                ? prohibitedScorers.get(0)
-                                : new MinShouldMatchSumScorer(weight, prohibitedScorers)));
-  }
-
-  @Override
-  public int docID() {
-    return doc;
-  }
-  
-  @Override
-  public int nextDoc() throws IOException {
-    return doc = countingSumScorer.nextDoc();
-  }
-  
-  @Override
-  public float score() throws IOException {
-    coordinator.nrMatchers = 0;
-    float sum = countingSumScorer.score();
-    return sum * coordinator.coordFactors[coordinator.nrMatchers];
-  }
-
-  @Override
-  public int freq() throws IOException {
-    return countingSumScorer.freq();
-  }
-
-  @Override
-  public int advance(int target) throws IOException {
-    return doc = countingSumScorer.advance(target);
-  }
-  
-  @Override
-  public long cost() {
-    return countingSumScorer.cost();
-  }
-
-  @Override
-  public Collection<ChildScorer> getChildren() {
-    ArrayList<ChildScorer> children = new ArrayList<>();
-    for (Scorer s : optionalScorers) {
-      children.add(new ChildScorer(s, "SHOULD"));
-    }
-    for (Scorer s : prohibitedScorers) {
-      children.add(new ChildScorer(s, "MUST_NOT"));
-    }
-    for (Scorer s : requiredScorers) {
-      children.add(new ChildScorer(s, "MUST"));
-    }
-    return children;
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
new file mode 100644
index 0000000..6edcf2c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java
@@ -0,0 +1,136 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+/** Internal document-at-a-time scorers used to deal with stupid coord() computation */
+class BooleanTopLevelScorers {
+  
+  /** 
+   * Used when there is more than one scorer in a query, but a segment
+   * only had one non-null scorer. This just wraps that scorer directly
+   * to factor in coord().
+   */
+  static class BoostedScorer extends FilterScorer {
+    private final float boost;
+    
+    BoostedScorer(Scorer in, float boost) {
+      super(in);
+      this.boost = boost;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return in.score() * boost;
+    }
+  }
+  
+  /** 
+   * Used when there are both mandatory and optional clauses, but minShouldMatch
+   * dictates that some of the optional clauses must match. The query is a conjunction,
+   * but must compute coord based on how many optional subscorers matched (freq).
+   */
+  static class CoordinatingConjunctionScorer extends ConjunctionScorer {
+    private final float coords[];
+    private final int reqCount;
+    private final Scorer req;
+    private final Scorer opt;
+    
+    CoordinatingConjunctionScorer(Weight weight, float coords[], Scorer req, int reqCount, Scorer opt) {
+      super(weight, new Scorer[] { req, opt });
+      this.coords = coords;
+      this.req = req;
+      this.reqCount = reqCount;
+      this.opt = opt;
+    }
+    
+    @Override
+    public float score() throws IOException {
+      return (req.score() + opt.score()) * coords[reqCount + opt.freq()];
+    }
+  }
+  
+  /** 
+   * Used when there are mandatory clauses with one optional clause: we compute
+   * coord based on whether the optional clause matched or not.
+   */
+  static class ReqSingleOptScorer extends ReqOptSumScorer {
+    // coord factor if just the required part matches
+    private final float coordReq;
+    // coord factor if both required and optional part matches 
+    private final float coordBoth;
+    
+    public ReqSingleOptScorer(Scorer reqScorer, Scorer optScorer, float coordReq, float coordBoth) {
+      super(reqScorer, optScorer);
+      this.coordReq = coordReq;
+      this.coordBoth = coordBoth;
+    }
+    
+    @Override
+    public float score() throws IOException {
+      int curDoc = reqScorer.docID();
+      float reqScore = reqScorer.score();
+      if (optScorer == null) {
+        return reqScore * coordReq;
+      }
+      
+      int optScorerDoc = optScorer.docID();
+      if (optScorerDoc < curDoc && (optScorerDoc = optScorer.advance(curDoc)) == NO_MORE_DOCS) {
+        optScorer = null;
+        return reqScore * coordReq;
+      }
+      
+      return optScorerDoc == curDoc ? (reqScore + optScorer.score()) * coordBoth : reqScore * coordReq;
+    }
+  }
+
+  /** 
+   * Used when there are mandatory clauses with optional clauses: we compute
+   * coord based on how many optional subscorers matched (freq).
+   */
+  static class ReqMultiOptScorer extends ReqOptSumScorer {
+    private final int requiredCount;
+    private final float coords[];
+    private final Scorer disjunction;
+    
+    public ReqMultiOptScorer(Scorer reqScorer, Scorer optScorer, int requiredCount, float coords[]) {
+      super(reqScorer, optScorer);
+      this.requiredCount = requiredCount;
+      this.coords = coords;
+      this.disjunction = optScorer;
+    }
+    
+    @Override
+    public float score() throws IOException {
+      int curDoc = reqScorer.docID();
+      float reqScore = reqScorer.score();
+      if (optScorer == null) {
+        return reqScore * coords[requiredCount];
+      }
+      
+      int optScorerDoc = optScorer.docID();
+      if (optScorerDoc < curDoc && (optScorerDoc = optScorer.advance(curDoc)) == NO_MORE_DOCS) {
+        optScorer = null;
+        return reqScore * coords[requiredCount];
+      }
+      
+      return optScorerDoc == curDoc ? (reqScore + optScorer.score()) * coords[requiredCount + disjunction.freq()] : reqScore * coords[requiredCount];
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
index 31c1d10..6f49e09 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
@@ -42,7 +42,7 @@
    * Organize subScorers into a min heap with scorers generating the earliest document on top.
    */
   protected final void heapify() {
-    for (int i = (numScorers >> 1) - 1; i >= 0; i--) {
+    for (int i = (numScorers >>> 1) - 1; i >= 0; i--) {
       heapAdjust(i);
     }
   }
@@ -55,7 +55,7 @@
     Scorer scorer = subScorers[root];
     int doc = scorer.docID();
     int i = root;
-    while (i <= (numScorers >> 1) - 1) {
+    while (i <= (numScorers >>> 1) - 1) {
       int lchild = (i << 1) + 1;
       Scorer lscorer = subScorers[lchild];
       int ldoc = lscorer.docID();
diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java b/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java
new file mode 100644
index 0000000..6b64d8a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java
@@ -0,0 +1,82 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.util.AttributeSource;
+
+/** 
+ * A {@code FilterScorer} contains another {@code Scorer}, which it
+ * uses as its basic source of data, possibly transforming the data along the
+ * way or providing additional functionality. The class
+ * {@code FilterScorer} itself simply implements all abstract methods
+ * of {@code Scorer} with versions that pass all requests to the
+ * contained scorer. Subclasses of {@code FilterScorer} may
+ * further override some of these methods and may also provide additional
+ * methods and fields.
+ */
+abstract class FilterScorer extends Scorer {
+  protected final Scorer in;
+  
+  public FilterScorer(Scorer in) {
+    super(in.weight);
+    this.in = in;
+  }
+  
+  @Override
+  public float score() throws IOException {
+    return in.score();
+  }
+
+  @Override
+  public int freq() throws IOException {
+    return in.freq();
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public Collection<ChildScorer> getChildren() {
+    return in.getChildren();
+  }
+
+  @Override
+  public AttributeSource attributes() {
+    return in.attributes();
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
index a4a2429..a2cb61b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
@@ -59,6 +59,8 @@
   /** The number of subscorers that provide the current match. */
   protected int nrMatchers = -1;
   private double score = Float.NaN;
+  
+  private final float coord[];
 
   /**
    * Construct a <code>MinShouldMatchSumScorer</code>.
@@ -72,7 +74,7 @@
    * <br>When minimumNrMatchers equals the number of subScorers,
    * it is more efficient to use <code>ConjunctionScorer</code>.
    */
-  public MinShouldMatchSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
+  public MinShouldMatchSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers, float coord[]) throws IOException {
     super(weight);
     this.nrInHeap = this.numScorers = subScorers.size();
 
@@ -105,17 +107,10 @@
     for (int i = 0; i < nrInHeap; i++) {
       this.subScorers[i] = this.sortedSubScorers[mm-1+i];
     }
+    this.coord = coord;
     minheapHeapify();
     assert minheapCheck();
   }
-  
-  /**
-   * Construct a <code>DisjunctionScorer</code>, using one as the minimum number
-   * of matching subscorers.
-   */
-  public MinShouldMatchSumScorer(Weight weight, List<Scorer> subScorers) throws IOException {
-    this(weight, subScorers, 1);
-  }
 
   @Override
   public final Collection<ChildScorer> getChildren() {
@@ -223,7 +218,7 @@
    */
   @Override
   public float score() throws IOException {
-    return (float) score;
+    return coord[nrMatchers] * (float) score;
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
index 14a3cf2..4e2a5f1 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
@@ -111,7 +111,7 @@
 
   @Override
   public Collection<ChildScorer> getChildren() {
-    return Collections.singleton(new ChildScorer(reqScorer, "FILTERED"));
+    return Collections.singleton(new ChildScorer(reqScorer, "MUST"));
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
index 7378e0a..d7b4d86 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
@@ -29,8 +29,8 @@
   /** The scorers passed from the constructor.
    * These are set to null as soon as their next() or skipTo() returns false.
    */
-  private Scorer reqScorer;
-  private Scorer optScorer;
+  protected Scorer reqScorer;
+  protected Scorer optScorer;
 
   /** Construct a <code>ReqOptScorer</code>.
    * @param reqScorer The required scorer. This must match.
diff --git a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
index 844290c..2ffe3d5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
@@ -32,23 +32,21 @@
  * several places, however all they have in hand is a {@link Scorer} object, and
  * might end up computing the score of a document more than once.
  */
-public class ScoreCachingWrappingScorer extends Scorer {
+public class ScoreCachingWrappingScorer extends FilterScorer {
 
-  private final Scorer scorer;
   private int curDoc = -1;
   private float curScore;
 
   /** Creates a new instance by wrapping the given scorer. */
   public ScoreCachingWrappingScorer(Scorer scorer) {
-    super(scorer.weight);
-    this.scorer = scorer;
+    super(scorer);
   }
 
   @Override
   public float score() throws IOException {
-    int doc = scorer.docID();
+    int doc = in.docID();
     if (doc != curDoc) {
-      curScore = scorer.score();
+      curScore = in.score();
       curDoc = doc;
     }
 
@@ -56,32 +54,7 @@
   }
 
   @Override
-  public int freq() throws IOException {
-    return scorer.freq();
-  }
-
-  @Override
-  public int docID() {
-    return scorer.docID();
-  }
-
-  @Override
-  public int nextDoc() throws IOException {
-    return scorer.nextDoc();
-  }
-  
-  @Override
-  public int advance(int target) throws IOException {
-    return scorer.advance(target);
-  }
-
-  @Override
   public Collection<ChildScorer> getChildren() {
-    return Collections.singleton(new ChildScorer(scorer, "CACHED"));
-  }
-
-  @Override
-  public long cost() {
-    return scorer.cost();
+    return Collections.singleton(new ChildScorer(in, "CACHED"));
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
index e6d8f34..c8e67a0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
@@ -1285,27 +1285,23 @@
     
     Document doc = new Document();
     doc.add(new StringField("id", "d0", Store.NO));
-    doc.add(new BinaryDocValuesField("f", toBytes(1L)));
+    doc.add(new BinaryDocValuesField("f1", toBytes(1L)));
+    doc.add(new BinaryDocValuesField("f2", toBytes(1L)));
     writer.addDocument(doc);
 
-    // create first gen of update files
-    writer.updateBinaryDocValue(new Term("id", "d0"), "f", toBytes(2L));
-    writer.commit();
-    int numFiles = dir.listAll().length;
-
-    DirectoryReader r = DirectoryReader.open(dir);
-    BytesRef scratch = new BytesRef();
-    assertEquals(2L, getValue(r.leaves().get(0).reader().getBinaryDocValues("f"), 0, scratch));
-    r.close();
-    
-    // create second gen of update files, first gen should be deleted
-    writer.updateBinaryDocValue(new Term("id", "d0"), "f", toBytes(5L));
-    writer.commit();
-    assertEquals(numFiles, dir.listAll().length);
-
-    r = DirectoryReader.open(dir);
-    assertEquals(5L, getValue(r.leaves().get(0).reader().getBinaryDocValues("f"), 0, scratch));
-    r.close();
+    // update each field twice to make sure all unneeded files are deleted
+    for (String f : new String[] { "f1", "f2" }) {
+      writer.updateBinaryDocValue(new Term("id", "d0"), f, toBytes(2L));
+      writer.commit();
+      int numFiles = dir.listAll().length;
+      
+      // update again, number of files shouldn't change (old field's gen is
+      // removed) 
+      writer.updateBinaryDocValue(new Term("id", "d0"), f, toBytes(3L));
+      writer.commit();
+      
+      assertEquals(numFiles, dir.listAll().length);
+    }
 
     writer.shutdown();
     dir.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
index 458fbe8..ea71775 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
@@ -44,7 +44,6 @@
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 
 
 /** JUnit adaptation of an older test case DocTest. */
@@ -240,7 +239,7 @@
         }
       }
 
-      return new SegmentCommitInfo(info, 0, -1L, -1L);
+      return new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
    }
 
 
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java
index f4d466f..a008716 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java
@@ -325,7 +325,7 @@
             segSeen.add(segName);
             SegmentInfo si = new Lucene46SegmentInfoFormat().getSegmentInfoReader().read(dir, segName, IOContext.DEFAULT);
             si.setCodec(codec);
-            SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1);
+            SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1);
             SegmentReader sr = new SegmentReader(sci, IOContext.DEFAULT);
             try {
               thread0Count += sr.docFreq(new Term("field", "threadID0"));
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
index 74d0308..72f45fb 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
@@ -1264,27 +1264,24 @@
     
     Document doc = new Document();
     doc.add(new StringField("id", "d0", Store.NO));
-    doc.add(new NumericDocValuesField("f", 1L));
+    doc.add(new NumericDocValuesField("f1", 1L));
+    doc.add(new NumericDocValuesField("f2", 1L));
     writer.addDocument(doc);
 
-    // create first gen of update files
-    writer.updateNumericDocValue(new Term("id", "d0"), "f", 2L);
-    writer.commit();
-    int numFiles = dir.listAll().length;
-
-    DirectoryReader r = DirectoryReader.open(dir);
-    assertEquals(2L, r.leaves().get(0).reader().getNumericDocValues("f").get(0));
-    r.close();
+    // update each field twice to make sure all unneeded files are deleted
+    for (String f : new String[] { "f1", "f2" }) {
+      writer.updateNumericDocValue(new Term("id", "d0"), f, 2L);
+      writer.commit();
+      int numFiles = dir.listAll().length;
+      
+      // update again, number of files shouldn't change (old field's gen is
+      // removed) 
+      writer.updateNumericDocValue(new Term("id", "d0"), f, 3L);
+      writer.commit();
+      
+      assertEquals(numFiles, dir.listAll().length);
+    }
     
-    // create second gen of update files, first gen should be deleted
-    writer.updateNumericDocValue(new Term("id", "d0"), "f", 5L);
-    writer.commit();
-    assertEquals(numFiles, dir.listAll().length);
-
-    r = DirectoryReader.open(dir);
-    assertEquals(5L, r.leaves().get(0).reader().getNumericDocValues("f").get(0));
-    r.close();
-
     writer.shutdown();
     dir.close();
   }
@@ -1455,7 +1452,6 @@
     writer.updateNumericDocValue(new Term("id", "doc-0"), "val", 100L);
     DirectoryReader reader = DirectoryReader.open(writer, true); // flush
     assertEquals(0, cachingDir.listCachedFiles().length);
-    for (String f : cachingDir.listAll()) System.out.println(f + " " + cachingDir.fileLength(f));
     
     IOUtils.close(reader, writer, cachingDir);
   }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
index ba45886..ed6ee45 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
@@ -30,7 +30,6 @@
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 
 public class TestSegmentMerger extends LuceneTestCase {
   //The variables for the new merged segment
@@ -91,7 +90,7 @@
     SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
                                                          new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged,
                                                                          false, codec, null),
-                                                         0, -1L, -1L),
+                                                         0, -1L, -1L, -1L),
                                                    newIOContext(random()));
     assertTrue(mergedReader != null);
     assertTrue(mergedReader.numDocs() == 2);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java
new file mode 100644
index 0000000..41b2fbe
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java
@@ -0,0 +1,752 @@
+package org.apache.lucene.search;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/** 
+ * Tests coord() computation by BooleanQuery
+ */
+public class TestBooleanCoord extends LuceneTestCase {
+  static Directory dir;
+  static DirectoryReader reader;
+  static IndexSearcher searcher;
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
+    
+    // we only add two documents for testing:
+    // the first document has 3 terms A,B,C (for positive matching). we test scores against this.
+    // the second document has 3 negative terms 1,2,3 that exist in the segment (for non-null scorers)
+    // to test terms that don't exist (null scorers), we use X,Y,Z
+    
+    Document doc = new Document();
+    doc.add(new StringField("field", "A", Field.Store.NO));
+    doc.add(new StringField("field", "B", Field.Store.NO));
+    doc.add(new StringField("field", "C", Field.Store.NO));
+    iw.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new StringField("field", "1", Field.Store.NO));
+    doc.add(new StringField("field", "2", Field.Store.NO));
+    doc.add(new StringField("field", "3", Field.Store.NO));
+    iw.addDocument(doc);
+
+    iw.shutdown();
+    reader = DirectoryReader.open(dir);
+    searcher = new IndexSearcher(reader);
+    // we set a similarity that just returns 1, the idea is to test coord
+    searcher.setSimilarity(new Similarity() {
+      @Override
+      public float coord(int overlap, int maxOverlap) {
+        return overlap / (float)maxOverlap;
+      }
+
+      @Override
+      public long computeNorm(FieldInvertState state) {
+        throw new AssertionError();
+      }
+
+      @Override
+      public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) {
+        return new SimWeight() {
+          @Override
+          public float getValueForNormalization() {
+            return 1f;
+          }
+
+          @Override
+          public void normalize(float queryNorm, float topLevelBoost) {}
+        };
+      }
+
+      @Override
+      public SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
+        return new SimScorer() {
+          @Override
+          public float score(int doc, float freq) {
+            return 1;
+          }
+
+          @Override
+          public float computeSlopFactor(int distance) {
+            throw new AssertionError();
+          }
+
+          @Override
+          public float computePayloadFactor(int doc, int start, int end, BytesRef payload) {
+            throw new AssertionError();
+          }
+        };
+      }
+    });
+  }
+  
+  @AfterClass
+  public static void afterClass() throws Exception {
+    reader.close();
+    reader = null;
+    dir.close();
+    dir = null;
+    searcher = null;
+  }
+  
+  // disjunctions
+  
+  public void testDisjunction1TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/1f, bq);
+  }
+  
+  public void testDisjunction2TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 1/1f, bq);
+  }
+  
+  public void testDisjunction1OutOf2() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testDisjunction1OutOf2Missing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testDisjunction1OutOf3() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testDisjunction1OutOf3MissingOne() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testDisjunction1OutOf3MissingTwo() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Y"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testDisjunction2OutOf3() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testDisjunction2OutOf3Missing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  // disjunctions with coord disabled
+  
+  public void testDisjunction1TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction2TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testDisjunction1OutOf2CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction1OutOf2MissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction1OutOf3CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction1OutOf3MissingOneCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction1OutOf3MissingTwoCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Y"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testDisjunction2OutOf3CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testDisjunction2OutOf3MissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  // minShouldMatch
+  public void testMinShouldMatch1TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/1f, bq);
+  }
+  
+  public void testMinShouldMatchn2TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 1/1f, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf2() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf2Missing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3MissingOne() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3MissingTwo() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Y"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf3() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf3Missing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf4() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/4f, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf4Missing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/4f, bq);
+  }
+  
+  // minShouldMatch with coord disabled
+  
+  public void testMinShouldMatch1TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch2TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf2CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf2MissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3MissingOneCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch1OutOf3MissingTwoCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Y"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf3CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf3MissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf4CoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMinShouldMatch2OutOf4MissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.SHOULD);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  // conjunctions
+  
+  public void testConjunction1TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    assertScore(1 * 1/1f, bq);
+  }
+  
+  public void testConjunction1TermMatches1Prohib() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.MUST_NOT);
+    assertScore(1 * 1/1f, bq);
+  }
+  
+  public void testConjunction1TermMatches2Prohib() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.MUST_NOT);
+    bq.add(term("2"), BooleanClause.Occur.MUST_NOT);
+    assertScore(1 * 1/1f, bq);
+  }
+  
+  public void testConjunction2TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.MUST);
+    assertScore(2 * 1/1f, bq);
+  }
+  
+  // conjunctions coord disabled
+  
+  public void testConjunction1TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    assertScore(1, bq);
+  }
+  
+  public void testConjunction1TermMatches1ProhibCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.MUST_NOT);
+    assertScore(1, bq);
+  }
+  
+  public void testConjunction1TermMatches2ProhibCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.MUST_NOT);
+    bq.add(term("2"), BooleanClause.Occur.MUST_NOT);
+    assertScore(1, bq);
+  }
+  
+  public void testConjunction2TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.MUST);
+    assertScore(2, bq);
+  }
+  
+  // optional + mandatory mix
+  public void testMix2TermMatches() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/2f, bq);
+  }
+  
+  public void testMixMatch1OutOfTwo() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testMixMatch1OutOfTwoMissing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/2f, bq);
+  }
+  
+  public void testMixMatch1OutOfThree() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMixMatch1OutOfThreeOneMissing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMixMatch2OutOfThree() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMixMatch2OutOfThreeMissing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMix2TermMatchesCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMixMatch1OutOfTwoCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMixMatch1OutOfTwoMissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMixMatch1OutOfThreeCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("2"), BooleanClause.Occur.SHOULD);
+    assertScore(1, bq);
+  }
+  
+  public void testMixMatch1OutOfThreeOneMissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(1 * 1/3f, bq);
+  }
+  
+  public void testMixMatch2OutOfThreeCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMixMatch2OutOfThreeMissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  // min should match + mandatory mix
+  
+  public void testMixMinShouldMatch2OutOfThree() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMixMinShouldMatch2OutOfThreeMissing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2 * 2/3f, bq);
+  }
+  
+  public void testMixMinShouldMatch3OutOfFour() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("C"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(3 * 3/4f, bq);
+  }
+  
+  public void testMixMinShouldMatch3OutOfFourMissing() throws Exception {
+    BooleanQuery bq = new BooleanQuery();
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("C"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(3 * 3/4f, bq);
+  }
+  
+  public void testMixMinShouldMatch2OutOfThreeCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMixMinShouldMatch2OutOfThreeMissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(1);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(2, bq);
+  }
+  
+  public void testMixMinShouldMatch3OutOfFourCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("C"), BooleanClause.Occur.SHOULD);
+    bq.add(term("1"), BooleanClause.Occur.SHOULD);
+    assertScore(3, bq);
+  }
+  
+  public void testMixMinShouldMatch3OutOfFourMissingCoordDisabled() throws Exception {
+    BooleanQuery bq = new BooleanQuery(true);
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(term("A"), BooleanClause.Occur.MUST);
+    bq.add(term("B"), BooleanClause.Occur.SHOULD);
+    bq.add(term("C"), BooleanClause.Occur.SHOULD);
+    bq.add(term("Z"), BooleanClause.Occur.SHOULD);
+    assertScore(3, bq);
+  }
+  
+  /** asserts score for our single matching good doc */
+  private void assertScore(final float expected, Query query) throws Exception {
+    // test in-order
+    Weight weight = searcher.createNormalizedWeight(query);
+    Scorer scorer = weight.scorer(reader.leaves().get(0), null);
+    assertTrue(scorer.docID() == -1 || scorer.docID() == DocIdSetIterator.NO_MORE_DOCS);
+    assertEquals(0, scorer.nextDoc());
+    assertEquals(expected, scorer.score(), 0.0001f);
+    
+    // test out-of-order (if supported)
+    if (weight.scoresDocsOutOfOrder()) {
+      final AtomicBoolean seen = new AtomicBoolean(false);
+      BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), false, null);
+      assertNotNull(bulkScorer);
+      bulkScorer.score(new LeafCollector() {
+        Scorer scorer;
+        
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          this.scorer = scorer;
+        }
+        
+        @Override
+        public void collect(int doc) throws IOException {
+          assertFalse(seen.get());
+          assertEquals(0, doc);
+          assertEquals(expected, scorer.score(), 0.0001f);
+          seen.set(true);
+        }
+
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return true;
+        }
+      }, 1);
+      assertTrue(seen.get());
+    }
+    
+    // test the explanation
+    Explanation expl = weight.explain(reader.leaves().get(0), 0);
+    assertEquals(expected, expl.getValue(), 0.0001f);
+  }
+  
+  private Query term(String s) {
+    return new TermQuery(new Term("field", s));
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
index a9c7cda..a129f2e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSubScorerFreqs.java
@@ -171,7 +171,8 @@
       boolean includeOptional = occur.contains("SHOULD");
       for (int i = 0; i < maxDocs; i++) {
         Map<Query, Float> doc0 = c.docCounts.get(i);
-        assertEquals(includeOptional ? 5 : 4, doc0.size());
+        // Y doesnt exist in the index, so its not in the scorer tree
+        assertEquals(4, doc0.size());
         assertEquals(1.0F, doc0.get(aQuery), FLOAT_TOLERANCE);
         assertEquals(4.0F, doc0.get(dQuery), FLOAT_TOLERANCE);
         if (includeOptional) {
@@ -179,7 +180,8 @@
         }
 
         Map<Query, Float> doc1 = c.docCounts.get(++i);
-        assertEquals(includeOptional ? 5 : 4, doc1.size());
+        // Y doesnt exist in the index, so its not in the scorer tree
+        assertEquals(4, doc1.size());
         assertEquals(1.0F, doc1.get(aQuery), FLOAT_TOLERANCE);
         assertEquals(1.0F, doc1.get(dQuery), FLOAT_TOLERANCE);
         if (includeOptional) {
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index a768774..f124829 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -448,7 +448,7 @@
 
       if (!fieldInfos.containsKey(fieldName)) {
         fieldInfos.put(fieldName, 
-            new FieldInfo(fieldName, true, fieldInfos.size(), false, false, false, this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS , null, null, null));
+            new FieldInfo(fieldName, true, fieldInfos.size(), false, false, false, this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS , null, null, -1, null));
       }
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
diff --git a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
index 5a3508d..a3c0287 100644
--- a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
+++ b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
@@ -140,7 +140,9 @@
       // Same info just changing the dir:
       SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(), 
                                             info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics());
-      destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen()));
+      destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
+          infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
+          infoPerCommit.getDocValuesGen()));
       // now copy files over
       Collection<String> files = infoPerCommit.files();
       for (final String srcName : files) {
diff --git a/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java b/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
index 7f5b673..de3eeb6 100644
--- a/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
+++ b/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
@@ -215,7 +215,7 @@
         }
       }
       filteredInfos.add(new FieldInfo(fi.name, fi.isIndexed(), fi.number, fi.hasVectors(), fi.omitsNorms(),
-                                      fi.hasPayloads(), fi.getIndexOptions(), type, fi.getNormType(), null));
+                                      fi.hasPayloads(), fi.getIndexOptions(), type, fi.getNormType(), -1, null));
     }
     fieldInfos = new FieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()]));
   }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
index 8a39cdb..6df153b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
@@ -357,7 +357,7 @@
 
       fieldInfoArray[fieldUpto] = new FieldInfo(field, true, fieldUpto, false, false, true,
                                                 IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
-                                                null, DocValuesType.NUMERIC, null);
+                                                null, DocValuesType.NUMERIC, -1, null);
       fieldUpto++;
 
       SortedMap<BytesRef,Long> postings = new TreeMap<>();
@@ -680,6 +680,7 @@
                                                    indexOptions,
                                                    null,
                                                    DocValuesType.NUMERIC,
+                                                   -1,
                                                    null);
     }
 
diff --git a/solr/core/src/java/org/apache/solr/search/Insanity.java b/solr/core/src/java/org/apache/solr/search/Insanity.java
index 2026b15..5b9f567 100644
--- a/solr/core/src/java/org/apache/solr/search/Insanity.java
+++ b/solr/core/src/java/org/apache/solr/search/Insanity.java
@@ -65,7 +65,7 @@
       for (FieldInfo fi : in.getFieldInfos()) {
         if (fi.name.equals(insaneField)) {
           filteredInfos.add(new FieldInfo(fi.name, fi.isIndexed(), fi.number, fi.hasVectors(), fi.omitsNorms(),
-                                          fi.hasPayloads(), fi.getIndexOptions(), null, fi.getNormType(), null));
+                                          fi.hasPayloads(), fi.getIndexOptions(), null, fi.getNormType(), -1, null));
         } else {
           filteredInfos.add(fi);
         }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 3259ced..6307bf6 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -1190,7 +1190,7 @@
 
     if (zkEnabled && isLeader && !isSubShardLeader)  {
       DocCollection coll = zkController.getClusterState().getCollection(collection);
-      List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
+      List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), cmd.getId(), null);
       // the list<node> will actually have only one element for an add request
       if (subShardLeaders != null && !subShardLeaders.isEmpty()) {
         ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 78d2e7b..dc8fd44 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -58,7 +58,7 @@
   
   // To prevent the test assertions firing too fast before cluster state
   // recognizes (and propagates) partitions
-  private static final long sleepMsBeforeHealPartition = 1000L;
+  private static final long sleepMsBeforeHealPartition = 2000L;
   
   private static final int maxWaitSecsToSeeAllActive = 30;
   
@@ -229,7 +229,7 @@
         ensureAllReplicasAreActive(testCollectionName, 3, maxWaitSecsToSeeAllActive);
     assertTrue("Expected 2 replicas for collection " + testCollectionName
         + " but found " + notLeaders.size() + "; clusterState: "
-        + cloudClient.getZkStateReader().getClusterState(),
+        + printClusterStateInfo(),
         notLeaders.size() == 2);
     
     sendDoc(1);
@@ -277,7 +277,7 @@
         ensureAllReplicasAreActive(testCollectionName, 3, maxWaitSecsToSeeAllActive);
     assertTrue("Expected 2 replicas for collection " + testCollectionName
         + " but found " + notLeaders.size() + "; clusterState: "
-        + cloudClient.getZkStateReader().getClusterState(),
+        + printClusterStateInfo(),
         notLeaders.size() == 2);
         
     sendDoc(1);
@@ -314,7 +314,8 @@
     Replica leader = 
         cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
     String leaderNode = leader.getNodeName();
-    assertNotNull("Could not find leader for shard1 of "+testCollectionName, leader);
+    assertNotNull("Could not find leader for shard1 of "+
+      testCollectionName+"; clusterState: "+printClusterStateInfo(), leader);
     JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
     
     // since maxShardsPerNode is 1, we're safe to kill the leader
@@ -343,16 +344,17 @@
       log.warn("No SocketProxy found for old leader node "+leaderNode);      
     }
 
-    Thread.sleep(sleepMsBeforeHealPartition);
+    Thread.sleep(10000); // give chance for new leader to be elected.
     
     Replica newLeader = 
         cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1", 60000);
         
-    assertNotNull("No new leader was elected after 60 seconds", newLeader);
+    assertNotNull("No new leader was elected after 60 seconds; clusterState: "+
+      printClusterStateInfo(),newLeader);
         
     assertTrue("Expected node "+shouldNotBeNewLeaderNode+
         " to NOT be the new leader b/c it was out-of-sync with the old leader! ClusterState: "+
-        cloudClient.getZkStateReader().getClusterState(), 
+        printClusterStateInfo(), 
         !shouldNotBeNewLeaderNode.equals(newLeader.getNodeName()));
     
     proxy0.reopen();
@@ -362,12 +364,19 @@
     cloudClient.getZkStateReader().updateClusterState(true);
     
     List<Replica> activeReps = getActiveOrRecoveringReplicas(testCollectionName, "shard1");
-    assertTrue("Expected 2 of 3 replicas to be active but only found "+activeReps.size()+"; "+activeReps, activeReps.size() == 2);
+    assertTrue("Expected 2 of 3 replicas to be active but only found "+
+      activeReps.size()+"; "+activeReps+"; clusterState: "+printClusterStateInfo(), 
+      activeReps.size() == 2);
         
     sendDoc(6);
     
     assertDocsExistInAllReplicas(activeReps, testCollectionName, 1, 6);
-  }  
+  }
+  
+  protected String printClusterStateInfo() throws Exception {
+    cloudClient.getZkStateReader().updateClusterState(true);
+    return String.valueOf(cloudClient.getZkStateReader().getClusterState());
+  }
   
   protected List<Replica> getActiveOrRecoveringReplicas(String testCollectionName, String shardId) throws Exception {    
     Map<String,Replica> activeReplicas = new HashMap<String,Replica>();    
@@ -492,10 +501,12 @@
     } // end while
     
     if (!allReplicasUp) 
-      fail("Didn't see all replicas come up within " + maxWaitMs + " ms! ClusterState: " + cs);
+      fail("Didn't see all replicas come up within " + maxWaitMs + 
+          " ms! ClusterState: " + printClusterStateInfo());
     
     if (notLeaders.isEmpty()) 
-      fail("Didn't isolate any replicas that are not the leader! ClusterState: " + cs);
+      fail("Didn't isolate any replicas that are not the leader! ClusterState: " + 
+         printClusterStateInfo());
     
     long diffMs = (System.currentTimeMillis() - startMs);
     log.info("Took " + diffMs + " ms to see all replicas become active.");
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
index df422dd..3c70c56 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplicationFactorTest.java
@@ -25,6 +25,7 @@
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
@@ -195,7 +196,8 @@
       NamedList resp = solrServer.request(up);
       NamedList hdr = (NamedList) resp.get("responseHeader");
       Integer batchRf = (Integer)hdr.get(UpdateRequest.REPFACT);
-      assertTrue("Expected rf="+expectedRf+" for batch but got "+batchRf, batchRf == expectedRf);      
+      assertTrue("Expected rf="+expectedRf+" for batch but got "+
+        batchRf+"; clusterState: "+printClusterStateInfo(), batchRf == expectedRf);      
     } finally {
       if (solrServer != null)
         solrServer.shutdown();
@@ -215,20 +217,20 @@
     
     List<Replica> replicas = 
         ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 10);
-    assertTrue("Expected active 2 replicas for "+testCollectionName, replicas.size() == 2);
+    assertTrue("Expected 2 active replicas for "+testCollectionName, replicas.size() == 2);
                 
     int rf = sendDoc(1, minRf);
-    assertTrue("Expected rf=3 as all replicas are up, but got "+rf, rf == 3);
+    assertRf(3, "all replicas should be active", rf);
         
     getProxyForReplica(replicas.get(0)).close();
     
     rf = sendDoc(2, minRf);
-    assertTrue("Expected rf=2 as one replica should be down, but got "+rf, rf == 2);
+    assertRf(2, "one replica should be down", rf);
 
     getProxyForReplica(replicas.get(1)).close();    
 
     rf = sendDoc(3, minRf);
-    assertTrue("Expected rf=1 as both replicas should be down, but got "+rf, rf == 1);
+    assertRf(1, "both replicas should be down", rf);
     
     // heal the partitions
     getProxyForReplica(replicas.get(0)).reopen();    
@@ -237,7 +239,7 @@
     ensureAllReplicasAreActive(testCollectionName, shardId, numShards, replicationFactor, 30);
     
     rf = sendDoc(4, minRf);
-    assertTrue("Expected rf=3 as partitions to replicas have been healed, but got "+rf, rf == 3);
+    assertRf(3, "partitions to replicas have been healed", rf);
     
     // now send a batch
     List<SolrInputDocument> batch = new ArrayList<SolrInputDocument>(10);
@@ -253,11 +255,11 @@
     up.add(batch);
     int batchRf = 
         cloudClient.getMinAchievedReplicationFactor(cloudClient.getDefaultCollection(), cloudClient.request(up)); 
-    assertTrue("Expected rf=3 for batch but got "+batchRf, batchRf == 3);
+    assertRf(3, "batch should have succeeded on all replicas", batchRf);
     
     // add some chaos to the batch
     getProxyForReplica(replicas.get(0)).close();
-
+    
     // now send a batch
     batch = new ArrayList<SolrInputDocument>(10);
     for (int i=15; i < 30; i++) {
@@ -272,7 +274,7 @@
     up.add(batch);
     batchRf = 
         cloudClient.getMinAchievedReplicationFactor(cloudClient.getDefaultCollection(), cloudClient.request(up)); 
-    assertTrue("Expected rf=2 for batch (one replica is down) but got "+batchRf, batchRf == 2);
+    assertRf(2, "batch should have succeeded on 2 replicas (only one replica should be down)", batchRf);
 
     // close the 2nd replica, and send a 3rd batch with expected achieved rf=1
     getProxyForReplica(replicas.get(1)).close();
@@ -290,11 +292,10 @@
     up.add(batch);
     batchRf = 
         cloudClient.getMinAchievedReplicationFactor(cloudClient.getDefaultCollection(), cloudClient.request(up)); 
-    assertTrue("Expected rf=1 for batch (two replicas are down) but got "+batchRf, batchRf == 1);
-    
+    assertRf(1, "batch should have succeeded on the leader only (both replicas should be down)", batchRf);
+
     getProxyForReplica(replicas.get(0)).reopen();        
     getProxyForReplica(replicas.get(1)).reopen();
-    Thread.sleep(1000);
   } 
     
   protected SocketProxy getProxyForReplica(Replica replica) throws Exception {
@@ -366,10 +367,10 @@
     } // end while
     
     if (!allReplicasUp) 
-      fail("Didn't see all replicas come up within " + maxWaitMs + " ms! ClusterState: " + cs);
+      fail("Didn't see all replicas come up within " + maxWaitMs + " ms! ClusterState: " + printClusterStateInfo());
     
     if (notLeaders.isEmpty()) 
-      fail("Didn't isolate any replicas that are not the leader! ClusterState: " + cs);
+      fail("Didn't isolate any replicas that are not the leader! ClusterState: " + printClusterStateInfo());
     
     long diffMs = (System.currentTimeMillis() - startMs);
     log.info("Took " + diffMs + " ms to see all replicas become active.");
@@ -378,4 +379,17 @@
     replicas.addAll(notLeaders.values());
     return replicas;
   }  
+  
+  protected void assertRf(int expected, String explain, int actual) throws Exception {
+    if (actual != expected) {
+      String assertionFailedMessage = 
+          String.format(Locale.ENGLISH, "Expected rf=%d because %s but got %d", expected, explain, actual);
+      fail(assertionFailedMessage+"; clusterState: "+printClusterStateInfo());
+    }    
+  }
+  
+  protected String printClusterStateInfo() throws Exception {
+    cloudClient.getZkStateReader().updateClusterState(true);
+    return String.valueOf(cloudClient.getZkStateReader().getClusterState());
+  }  
 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
index 0adc8f0..d1c572d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
@@ -38,7 +38,6 @@
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.update.DirectUpdateHandler2;
 import org.junit.After;
 import org.junit.Before;
 
@@ -52,14 +51,12 @@
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.lucene.util.LuceneTestCase.BadApple;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
 import static org.apache.solr.cloud.OverseerCollectionProcessor.REPLICATION_FACTOR;
 
 @Slow
-@BadApple(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5309")
 public class ShardSplitTest extends BasicDistributedZkTest {
 
   public static final String SHARD1_0 = SHARD1 + "_0";
@@ -235,7 +232,7 @@
       }
     }
 
-    waitForRecoveriesToFinish(false);
+    waitForRecoveriesToFinish(true);
     checkDocCountsAndShardStates(docCounts, numReplicas);
   }
 
@@ -466,8 +463,8 @@
 
     logDebugHelp(docCounts, response, shard10Count, response2, shard11Count);
 
-    assertEquals("Wrong doc count on shard1_0", docCounts[0], shard10Count);
-    assertEquals("Wrong doc count on shard1_1", docCounts[1], shard11Count);
+    assertEquals("Wrong doc count on shard1_0. See SOLR-5309", docCounts[0], shard10Count);
+    assertEquals("Wrong doc count on shard1_1. See SOLR-5309", docCounts[1], shard11Count);
   }
 
   protected void checkSubShardConsistency(String shard) throws SolrServerException {