You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/09/21 13:42:15 UTC

[11/16] lucene-solr:master: LUCENE-7407: switch doc values usage to an iterator API, based on DocIdSetIterator, instead of random acces, freeing codecs for future improvements

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
index 735dfbe..b638a29 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
@@ -25,7 +25,6 @@ import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
@@ -35,6 +34,10 @@ import org.apache.lucene.util.BytesRefBuilder;
  * TopFieldCollector}.  The concrete public FieldComparator
  * classes here correspond to the SortField types.
  *
+ * <p>The document IDs passed to these methods must only
+ * move forwards, since they are using doc values iterators
+ * to retrieve sort values.</p>
+ *
  * <p>This API is designed to achieve high performance
  * sorting, by exposing a tight interaction with {@link
  * FieldValueHitQueue} as it visits hits.  Whenever a hit is
@@ -140,7 +143,6 @@ public abstract class FieldComparator<T> {
   public static abstract class NumericComparator<T extends Number> extends SimpleFieldComparator<T> {
     protected final T missingValue;
     protected final String field;
-    protected Bits docsWithField;
     protected NumericDocValues currentReaderValues;
     
     public NumericComparator(String field, T missingValue) {
@@ -151,26 +153,12 @@ public abstract class FieldComparator<T> {
     @Override
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
       currentReaderValues = getNumericDocValues(context, field);
-      if (missingValue != null) {
-        docsWithField = getDocsWithValue(context, field);
-        // optimization to remove unneeded checks on the bit interface:
-        if (docsWithField instanceof Bits.MatchAllBits) {
-          docsWithField = null;
-        }
-      } else {
-        docsWithField = null;
-      }
     }
     
     /** Retrieves the NumericDocValues for the field in this segment */
     protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
       return DocValues.getNumeric(context.reader(), field);
     }
-
-    /** Retrieves a {@link Bits} instance representing documents that have a value in this segment. */
-    protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
-      return DocValues.getDocsWithField(context.reader(), field);
-    }
   }
 
   /** Parses field's values as double (using {@link
@@ -182,40 +170,38 @@ public abstract class FieldComparator<T> {
 
     /** 
      * Creates a new comparator based on {@link Double#compare} for {@code numHits}.
-     * When a document has no value for the field, {@code missingValue} is substituted. 
+     * When a document has no value for the field, {@code missingValue} is substituted.
      */
     public DoubleComparator(int numHits, String field, Double missingValue) {
-      super(field, missingValue);
+      super(field, missingValue != null ? missingValue : 0.0);
       values = new double[numHits];
     }
 
+    private double getValueForDoc(int doc) throws IOException {
+      int curDocID = currentReaderValues.docID();
+      if (doc > curDocID) {
+        curDocID = currentReaderValues.advance(doc);
+      }
+      if (doc == curDocID) {
+        return Double.longBitsToDouble(currentReaderValues.longValue());
+      } else {
+        return missingValue;
+      }
+    }
+
     @Override
     public int compare(int slot1, int slot2) {
       return Double.compare(values[slot1], values[slot2]);
     }
 
     @Override
-    public int compareBottom(int doc) {
-      double v2 = Double.longBitsToDouble(currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Double.compare(bottom, v2);
+    public int compareBottom(int doc) throws IOException {
+      return Double.compare(bottom, getValueForDoc(doc));
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      double v2 = Double.longBitsToDouble(currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
+    public void copy(int slot, int doc) throws IOException {
+      values[slot] = getValueForDoc(doc);
     }
     
     @Override
@@ -234,14 +220,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
-      double docValue = Double.longBitsToDouble(currentReaderValues.get(doc));
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Double.compare(topValue, docValue);
+    public int compareTop(int doc) throws IOException {
+      return Double.compare(topValue, getValueForDoc(doc));
     }
   }
 
@@ -257,38 +237,35 @@ public abstract class FieldComparator<T> {
      * When a document has no value for the field, {@code missingValue} is substituted. 
      */
     public FloatComparator(int numHits, String field, Float missingValue) {
-      super(field, missingValue);
+      super(field, missingValue != null ? missingValue : 0.0f);
       values = new float[numHits];
     }
     
+    private float getValueForDoc(int doc) throws IOException {
+      int curDocID = currentReaderValues.docID();
+      if (doc > curDocID) {
+        curDocID = currentReaderValues.advance(doc);
+      }
+      if (doc == curDocID) {
+        return Float.intBitsToFloat((int) currentReaderValues.longValue());
+      } else {
+        return missingValue;
+      }
+    }
+
     @Override
     public int compare(int slot1, int slot2) {
       return Float.compare(values[slot1], values[slot2]);
     }
 
     @Override
-    public int compareBottom(int doc) {
-      // TODO: are there sneaky non-branch ways to compute sign of float?
-      float v2 = Float.intBitsToFloat((int)currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Float.compare(bottom, v2);
+    public int compareBottom(int doc) throws IOException {
+      return Float.compare(bottom, getValueForDoc(doc));
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      float v2 =  Float.intBitsToFloat((int)currentReaderValues.get(doc));
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
+    public void copy(int slot, int doc) throws IOException {
+      values[slot] = getValueForDoc(doc);
     }
     
     @Override
@@ -307,14 +284,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
-      float docValue = Float.intBitsToFloat((int)currentReaderValues.get(doc));
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Float.compare(topValue, docValue);
+    public int compareTop(int doc) throws IOException {
+      return Float.compare(topValue, getValueForDoc(doc));
     }
   }
 
@@ -330,9 +301,23 @@ public abstract class FieldComparator<T> {
      * When a document has no value for the field, {@code missingValue} is substituted. 
      */
     public IntComparator(int numHits, String field, Integer missingValue) {
-      super(field, missingValue);
+      super(field, missingValue != null ? missingValue : 0);
+      //System.out.println("IntComparator.init");
+      //new Throwable().printStackTrace(System.out);
       values = new int[numHits];
     }
+
+    private int getValueForDoc(int doc) throws IOException {
+      int curDocID = currentReaderValues.docID();
+      if (doc > curDocID) {
+        curDocID = currentReaderValues.advance(doc);
+      }
+      if (doc == curDocID) {
+        return (int) currentReaderValues.longValue();
+      } else {
+        return missingValue;
+      }
+    }
         
     @Override
     public int compare(int slot1, int slot2) {
@@ -340,27 +325,13 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareBottom(int doc) {
-      int v2 = (int) currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Integer.compare(bottom, v2);
+    public int compareBottom(int doc) throws IOException {
+      return Integer.compare(bottom, getValueForDoc(doc));
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      int v2 = (int) currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
+    public void copy(int slot, int doc) throws IOException {
+      values[slot] = getValueForDoc(doc);
     }
     
     @Override
@@ -379,14 +350,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
-      int docValue = (int) currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Integer.compare(topValue, docValue);
+    public int compareTop(int doc) throws IOException {
+      return Integer.compare(topValue, getValueForDoc(doc));
     }
   }
 
@@ -402,39 +367,35 @@ public abstract class FieldComparator<T> {
      * When a document has no value for the field, {@code missingValue} is substituted. 
      */
     public LongComparator(int numHits, String field, Long missingValue) {
-      super(field, missingValue);
+      super(field, missingValue != null ? missingValue : 0L);
       values = new long[numHits];
     }
 
+    private long getValueForDoc(int doc) throws IOException {
+      int curDocID = currentReaderValues.docID();
+      if (doc > curDocID) {
+        curDocID = currentReaderValues.advance(doc);
+      }
+      if (doc == curDocID) {
+        return currentReaderValues.longValue();
+      } else {
+        return missingValue;
+      }
+    }
+
     @Override
     public int compare(int slot1, int slot2) {
       return Long.compare(values[slot1], values[slot2]);
     }
 
     @Override
-    public int compareBottom(int doc) {
-      // TODO: there are sneaky non-branch ways to compute
-      // -1/+1/0 sign
-      long v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Long.compare(bottom, v2);
+    public int compareBottom(int doc) throws IOException {
+      return Long.compare(bottom, getValueForDoc(doc));
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      long v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
+    public void copy(int slot, int doc) throws IOException {
+      values[slot] = getValueForDoc(doc);
     }
     
     @Override
@@ -453,14 +414,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
-      long docValue = currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Long.compare(topValue, docValue);
+    public int compareTop(int doc) throws IOException {
+      return Long.compare(topValue, getValueForDoc(doc));
     }
   }
 
@@ -700,6 +655,18 @@ public abstract class FieldComparator<T> {
       }
     }
 
+    private int getOrdForDoc(int doc) throws IOException {
+      int curDocID = termsIndex.docID();
+      if (doc > curDocID) {
+        if (termsIndex.advance(doc) == doc) {
+          return termsIndex.ordValue();
+        }
+      } else if (doc == curDocID) {
+        return termsIndex.ordValue();
+      }
+      return -1;
+    }
+
     @Override
     public int compare(int slot1, int slot2) {
       if (readerGen[slot1] == readerGen[slot2]) {
@@ -720,9 +687,9 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareBottom(int doc) {
+    public int compareBottom(int doc) throws IOException {
       assert bottomSlot != -1;
-      int docOrd = termsIndex.getOrd(doc);
+      int docOrd = getOrdForDoc(doc);
       if (docOrd == -1) {
         docOrd = missingOrd;
       }
@@ -740,8 +707,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      int ord = termsIndex.getOrd(doc);
+    public void copy(int slot, int doc) throws IOException {
+      int ord = getOrdForDoc(doc);
       if (ord == -1) {
         ord = missingOrd;
         values[slot] = null;
@@ -836,9 +803,9 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
+    public int compareTop(int doc) throws IOException {
 
-      int ord = termsIndex.getOrd(doc);
+      int ord = getOrdForDoc(doc);
       if (ord == -1) {
         ord = missingOrd;
       }
@@ -883,7 +850,6 @@ public abstract class FieldComparator<T> {
     private final BytesRef[] values;
     private final BytesRefBuilder[] tempBRs;
     private BinaryDocValues docTerms;
-    private Bits docsWithField;
     private final String field;
     private BytesRef bottom;
     private BytesRef topValue;
@@ -897,6 +863,18 @@ public abstract class FieldComparator<T> {
       missingSortCmp = sortMissingLast ? 1 : -1;
     }
 
+    private BytesRef getValueForDoc(int doc) throws IOException {
+      int curDocID = docTerms.docID();
+      if (doc > curDocID) {
+        curDocID = docTerms.advance(doc);
+      }
+      if (doc == curDocID) {
+        return docTerms.binaryValue();
+      } else {
+        return null;
+      }
+    }
+
     @Override
     public int compare(int slot1, int slot2) {
       final BytesRef val1 = values[slot1];
@@ -905,14 +883,14 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareBottom(int doc) {
-      final BytesRef comparableBytes = getComparableBytes(doc, docTerms.get(doc));
+    public int compareBottom(int doc) throws IOException {
+      final BytesRef comparableBytes = getValueForDoc(doc);
       return compareValues(bottom, comparableBytes);
     }
 
     @Override
-    public void copy(int slot, int doc) {
-      final BytesRef comparableBytes = getComparableBytes(doc, docTerms.get(doc));
+    public void copy(int slot, int doc) throws IOException {
+      final BytesRef comparableBytes = getValueForDoc(doc);
       if (comparableBytes == null) {
         values[slot] = null;
       } else {
@@ -929,27 +907,17 @@ public abstract class FieldComparator<T> {
       return DocValues.getBinary(context.reader(), field);
     }
 
-    /** Retrieves the set of documents that have a value in this segment */
-    protected Bits getDocsWithField(LeafReaderContext context, String field) throws IOException {
-      return DocValues.getDocsWithField(context.reader(), field);
-    }
-
     /** Check whether the given value represents <tt>null</tt>. This can be
      *  useful if the {@link BinaryDocValues} returned by {@link #getBinaryDocValues}
-     *  use a special value as a sentinel. The default implementation checks
-     *  {@link #getDocsWithField}.
+     *  use a special value as a sentinel.
      *  <p>NOTE: The null value can only be an EMPTY {@link BytesRef}. */
-    protected boolean isNull(int doc, BytesRef term) {
-      return docsWithField != null && docsWithField.get(doc) == false;
+    protected boolean isNull(int doc, BytesRef term) throws IOException {
+      return getValueForDoc(doc) == null;
     }
 
     @Override
     public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
       docTerms = getBinaryDocValues(context, field);
-      docsWithField = getDocsWithField(context, field);
-      if (docsWithField instanceof Bits.MatchAllBits) {
-        docsWithField = null;
-      }
       return this;
     }
     
@@ -985,20 +953,8 @@ public abstract class FieldComparator<T> {
     }
 
     @Override
-    public int compareTop(int doc) {
-      final BytesRef comparableBytes = getComparableBytes(doc, docTerms.get(doc));
-      return compareValues(topValue, comparableBytes);
-    }
-
-    /**
-     * Given a document and a term, return the term itself if it exists or
-     * <tt>null</tt> otherwise.
-     */
-    private BytesRef getComparableBytes(int doc, BytesRef term) {
-      if (term.length == 0 && isNull(doc, term)) {
-        return null;
-      }
-      return term;
+    public int compareTop(int doc) throws IOException {
+      return compareValues(topValue, getValueForDoc(doc));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java b/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
index 42fc256..8a47eea 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
@@ -20,13 +20,15 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Objects;
 
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.Bits;
 
 /**
  * A {@link Query} that matches documents that have a value for a given field
- * as reported by {@link LeafReader#getDocsWithField(String)}.
+ * as reported by doc values iterators.
  */
 public final class FieldValueQuery extends Query {
 
@@ -60,14 +62,41 @@ public final class FieldValueQuery extends Query {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new RandomAccessWeight(this, boost) {
-
+    return new ConstantScoreWeight(this, boost) {
       @Override
-      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
-        return context.reader().getDocsWithField(field);
-      }
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        FieldInfos fieldInfos = context.reader().getFieldInfos();
+        FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        if (fieldInfo == null) {
+          return null;
+        }
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        LeafReader reader = context.reader();
+        DocIdSetIterator iterator;
+        switch(dvType) {
+        case NONE:
+          return null;
+        case NUMERIC:
+          iterator = reader.getNumericDocValues(field);
+          break;
+        case BINARY:
+          iterator = reader.getBinaryDocValues(field);
+          break;
+        case SORTED:
+          iterator = reader.getSortedDocValues(field);
+          break;
+        case SORTED_NUMERIC:
+          iterator = reader.getSortedNumericDocValues(field);
+          break;
+        case SORTED_SET:
+          iterator = reader.getSortedSetDocValues(field);
+          break;
+        default:
+          throw new AssertionError();
+        }
 
+        return new ConstantScoreScorer(this, score(), iterator);
+      }
     };
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
index acc8282..8165204 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
@@ -553,7 +553,7 @@ final class SloppyPhraseScorer extends Scorer {
   }
   
   @Override
-  public float score() {
+  public float score() throws IOException {
     return docScorer.score(docID(), sloppyFreq);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
index 314cc4f..56626e8 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java
@@ -17,7 +17,10 @@
 package org.apache.lucene.search;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FilterNumericDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.util.NumericUtils;
@@ -78,17 +81,17 @@ public class SortedNumericSelector {
     // undo the numericutils sortability
     switch(numericType) {
       case FLOAT:
-        return new NumericDocValues() {
+        return new FilterNumericDocValues(view) {
           @Override
-          public long get(int docID) {
-            return NumericUtils.sortableFloatBits((int) view.get(docID));
+          public long longValue() {
+            return NumericUtils.sortableFloatBits((int) in.longValue());
           }
         };
       case DOUBLE:
-        return new NumericDocValues() {
+        return new FilterNumericDocValues(view) {
           @Override
-          public long get(int docID) {
-            return NumericUtils.sortableDoubleBits(view.get(docID));
+          public long longValue() {
+            return NumericUtils.sortableDoubleBits(in.longValue());
           }
         };
       default:
@@ -99,39 +102,93 @@ public class SortedNumericSelector {
   /** Wraps a SortedNumericDocValues and returns the first value (min) */
   static class MinValue extends NumericDocValues {
     final SortedNumericDocValues in;
+    private long value;
     
     MinValue(SortedNumericDocValues in) {
       this.in = in;
     }
 
     @Override
-    public long get(int docID) {
-      in.setDocument(docID);
-      if (in.count() == 0) {
-        return 0; // missing
-      } else {
-        return in.valueAt(0);
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int docID = in.nextDoc();
+      if (docID != NO_MORE_DOCS) {
+        value = in.nextValue();
       }
+      return docID;
     }
-  }
-  
+
+    @Override
+    public int advance(int target) throws IOException {
+      int docID = in.advance(target);
+      if (docID != NO_MORE_DOCS) {
+        value = in.nextValue();
+      }
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+
+    @Override
+    public long longValue() {
+      return value;
+    }
+  }    
+
   /** Wraps a SortedNumericDocValues and returns the last value (max) */
   static class MaxValue extends NumericDocValues {
     final SortedNumericDocValues in;
+    private long value;
     
     MaxValue(SortedNumericDocValues in) {
       this.in = in;
     }
 
     @Override
-    public long get(int docID) {
-      in.setDocument(docID);
-      final int count = in.count();
-      if (count == 0) {
-        return 0; // missing
-      } else {
-        return in.valueAt(count-1);
+    public int docID() {
+      return in.docID();
+    }
+
+    private void setValue() throws IOException {
+      int count = in.docValueCount();
+      for(int i=0;i<count;i++) {
+        value = in.nextValue();
       }
     }
-  }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int docID = in.nextDoc();
+      if (docID != NO_MORE_DOCS) {
+        setValue();
+      }
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int docID = in.advance(target);
+      if (docID != NO_MORE_DOCS) {
+        setValue();
+      }
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+
+    @Override
+    public long longValue() {
+      return value;
+    }
+  }    
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
index 188a408..22d7457 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/SortedSetSelector.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedSetSelector.java b/lucene/core/src/java/org/apache/lucene/search/SortedSetSelector.java
index dee5e26..25fa8d3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedSetSelector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedSetSelector.java
@@ -17,12 +17,16 @@
 package org.apache.lucene.search;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.RandomAccessOrds;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
 /** Selects a value from the document's set to use as the representative value */
 public class SortedSetSelector {
   
@@ -73,18 +77,12 @@ public class SortedSetSelector {
       // so just sort on the underlying single-valued dv directly.
       // regardless of selector type, this optimization is safe!
       return singleton;
-    } else if (selector == Type.MIN) {
-      return new MinValue(sortedSet);
     } else {
-      if (sortedSet instanceof RandomAccessOrds == false) {
-        throw new UnsupportedOperationException("codec does not support random access ordinals, cannot use selector: " + selector + " docValsImpl: " + sortedSet.toString());
-      }
-      RandomAccessOrds randomOrds = (RandomAccessOrds) sortedSet;
       switch(selector) {
-        case MAX: return new MaxValue(randomOrds);
-        case MIDDLE_MIN: return new MiddleMinValue(randomOrds);
-        case MIDDLE_MAX: return new MiddleMaxValue(randomOrds);
-        case MIN: 
+        case MIN: return new MinValue(sortedSet);
+        case MAX: return new MaxValue(sortedSet);
+        case MIDDLE_MIN: return new MiddleMinValue(sortedSet);
+        case MIDDLE_MAX: return new MiddleMaxValue(sortedSet);
         default: 
           throw new AssertionError();
       }
@@ -94,15 +92,39 @@ public class SortedSetSelector {
   /** Wraps a SortedSetDocValues and returns the first ordinal (min) */
   static class MinValue extends SortedDocValues {
     final SortedSetDocValues in;
+    private int ord;
     
     MinValue(SortedSetDocValues in) {
       this.in = in;
     }
 
     @Override
-    public int getOrd(int docID) {
-      in.setDocument(docID);
-      return (int) in.nextOrd();
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      in.nextDoc();
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      in.advance(target);
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+    
+    @Override
+    public int ordValue() {
+      return ord;
     }
 
     @Override
@@ -119,25 +141,52 @@ public class SortedSetSelector {
     public int lookupTerm(BytesRef key) {
       return (int) in.lookupTerm(key);
     }
+
+    private void setOrd() throws IOException {
+      if (docID() != NO_MORE_DOCS) {
+        ord = (int) in.nextOrd();
+      } else {
+        ord = (int) NO_MORE_ORDS;
+      }
+    }
   }
   
   /** Wraps a SortedSetDocValues and returns the last ordinal (max) */
   static class MaxValue extends SortedDocValues {
-    final RandomAccessOrds in;
+    final SortedSetDocValues in;
+    private int ord;
     
-    MaxValue(RandomAccessOrds in) {
+    MaxValue(SortedSetDocValues in) {
       this.in = in;
     }
 
     @Override
-    public int getOrd(int docID) {
-      in.setDocument(docID);
-      final int count = in.cardinality();
-      if (count == 0) {
-        return -1;
-      } else {
-        return (int) in.ordAt(count-1);
-      }
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      in.nextDoc();
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      in.advance(target);
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+    
+    @Override
+    public int ordValue() {
+      return ord;
     }
 
     @Override
@@ -149,30 +198,64 @@ public class SortedSetSelector {
     public int getValueCount() {
       return (int) in.getValueCount();
     }
-    
+
     @Override
     public int lookupTerm(BytesRef key) {
       return (int) in.lookupTerm(key);
     }
+
+    private void setOrd() throws IOException {
+      if (docID() != NO_MORE_DOCS) {
+        while(true) {
+          long nextOrd = in.nextOrd();
+          if (nextOrd == NO_MORE_ORDS) {
+            break;
+          }
+          ord = (int) nextOrd;
+        }
+      } else {
+        ord = (int) NO_MORE_ORDS;
+      }
+    }
   }
   
   /** Wraps a SortedSetDocValues and returns the middle ordinal (or min of the two) */
   static class MiddleMinValue extends SortedDocValues {
-    final RandomAccessOrds in;
+    final SortedSetDocValues in;
+    private int ord;
+    private int[] ords = new int[8];
     
-    MiddleMinValue(RandomAccessOrds in) {
+    MiddleMinValue(SortedSetDocValues in) {
       this.in = in;
     }
 
     @Override
-    public int getOrd(int docID) {
-      in.setDocument(docID);
-      final int count = in.cardinality();
-      if (count == 0) {
-        return -1;
-      } else {
-        return (int) in.ordAt((count-1) >>> 1);
-      }
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      in.nextDoc();
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      in.advance(target);
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+    
+    @Override
+    public int ordValue() {
+      return ord;
     }
 
     @Override
@@ -184,30 +267,76 @@ public class SortedSetSelector {
     public int getValueCount() {
       return (int) in.getValueCount();
     }
-    
+
     @Override
     public int lookupTerm(BytesRef key) {
       return (int) in.lookupTerm(key);
     }
+
+    private void setOrd() throws IOException {
+      if (docID() != NO_MORE_DOCS) {
+        int upto = 0;
+        while (true) {
+          long nextOrd = in.nextOrd();
+          if (nextOrd == NO_MORE_ORDS) {
+            break;
+          }
+          if (upto == ords.length) {
+            ords = ArrayUtil.grow(ords);
+          }
+          ords[upto++] = (int) nextOrd;
+        }
+
+        if (upto == 0) {
+          // iterator should not have returned this docID if it has no ords:
+          assert false;
+          ord = (int) NO_MORE_ORDS;
+        } else {
+          ord = ords[(upto-1) >>> 1];
+        }
+      } else {
+        ord = (int) NO_MORE_ORDS;
+      }
+    }
   }
   
   /** Wraps a SortedSetDocValues and returns the middle ordinal (or max of the two) */
   static class MiddleMaxValue extends SortedDocValues {
-    final RandomAccessOrds in;
+    final SortedSetDocValues in;
+    private int ord;
+    private int[] ords = new int[8];
     
-    MiddleMaxValue(RandomAccessOrds in) {
+    MiddleMaxValue(SortedSetDocValues in) {
       this.in = in;
     }
 
     @Override
-    public int getOrd(int docID) {
-      in.setDocument(docID);
-      final int count = in.cardinality();
-      if (count == 0) {
-        return -1;
-      } else {
-        return (int) in.ordAt(count >>> 1);
-      }
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      in.nextDoc();
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      in.advance(target);
+      setOrd();
+      return docID();
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+    
+    @Override
+    public int ordValue() {
+      return ord;
     }
 
     @Override
@@ -219,10 +348,36 @@ public class SortedSetSelector {
     public int getValueCount() {
       return (int) in.getValueCount();
     }
-    
+
     @Override
     public int lookupTerm(BytesRef key) {
       return (int) in.lookupTerm(key);
     }
+
+    private void setOrd() throws IOException {
+      if (docID() != NO_MORE_DOCS) {
+        int upto = 0;
+        while (true) {
+          long nextOrd = in.nextOrd();
+          if (nextOrd == NO_MORE_ORDS) {
+            break;
+          }
+          if (upto == ords.length) {
+            ords = ArrayUtil.grow(ords);
+          }
+          ords[upto++] = (int) nextOrd;
+        }
+
+        if (upto == 0) {
+          // iterator should not have returned this docID if it has no ords:
+          assert false;
+          ord = (int) NO_MORE_ORDS;
+        } else {
+          ord = ords[upto >>> 1];
+        }
+      } else {
+        ord = (int) NO_MORE_ORDS;
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java b/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
index 5f81e85..da2546f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 
@@ -122,8 +122,7 @@ public class SortedSetSortField extends SortField {
     return new FieldComparator.TermOrdValComparator(numHits, getField(), missingValue == STRING_LAST) {
       @Override
       protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
-        SortedSetDocValues sortedSet = DocValues.getSortedSet(context.reader(), field);
-        return SortedSetSelector.wrap(sortedSet, selector);
+        return SortedSetSelector.wrap(DocValues.getSortedSet(context.reader(), field), selector);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/package-info.java b/lucene/core/src/java/org/apache/lucene/search/package-info.java
index b4d4780..fc0ab2e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/search/package-info.java
@@ -309,7 +309,7 @@
  * <p>
  * Finally, you can extend the low level {@link org.apache.lucene.search.similarities.Similarity Similarity} directly
  * to implement a new retrieval model, or to use external scoring factors particular to your application. For example,
- * a custom Similarity can access per-document values via {@link org.apache.lucene.index.NumericDocValues} and 
+ * a custom Similarity can access per-document values via {@link org.apache.lucene.index.NumericDocValues} and
  * integrate them into the score.
  * <p>
  * See the {@link org.apache.lucene.search.similarities} package documentation for information

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
index cbce18e..0cb86db 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
@@ -238,14 +238,27 @@ public class BM25Similarity extends Similarity {
     }
     
     @Override
-    public float score(int doc, float freq) {
+    public float score(int doc, float freq) throws IOException {
       // if there are no norms, we act as if b=0
-      float norm = norms == null ? k1 : cache[(byte)norms.get(doc) & 0xFF];
+      float norm;
+      if (norms == null) {
+        norm = k1;
+      } else {
+        int normsDocID = norms.docID();
+        if (normsDocID < doc) {
+          normsDocID = norms.advance(doc);
+        }
+        if (normsDocID == doc) {
+          norm = cache[(byte)norms.longValue() & 0xFF];
+        } else {
+          norm = cache[0];
+        }
+      }
       return weightValue * freq / (freq + norm);
     }
     
     @Override
-    public Explanation explain(int doc, Explanation freq) {
+    public Explanation explain(int doc, Explanation freq) throws IOException {
       return explainScore(doc, freq, stats, norms);
     }
 
@@ -286,7 +299,7 @@ public class BM25Similarity extends Similarity {
 
   }
 
-  private Explanation explainTFNorm(int doc, Explanation freq, BM25Stats stats, NumericDocValues norms) {
+  private Explanation explainTFNorm(int doc, Explanation freq, BM25Stats stats, NumericDocValues norms) throws IOException {
     List<Explanation> subs = new ArrayList<>();
     subs.add(freq);
     subs.add(Explanation.match(k1, "parameter k1"));
@@ -296,7 +309,13 @@ public class BM25Similarity extends Similarity {
           (freq.getValue() * (k1 + 1)) / (freq.getValue() + k1),
           "tfNorm, computed from:", subs);
     } else {
-      float doclen = decodeNormValue((byte)norms.get(doc));
+      byte norm;
+      if (norms.advance(doc) == doc) {
+        norm = (byte) norms.longValue();
+      } else {
+        norm = 0;
+      }
+      float doclen = decodeNormValue(norm);
       subs.add(Explanation.match(b, "parameter b"));
       subs.add(Explanation.match(stats.avgdl, "avgFieldLength"));
       subs.add(Explanation.match(doclen, "fieldLength"));
@@ -306,7 +325,7 @@ public class BM25Similarity extends Similarity {
     }
   }
 
-  private Explanation explainScore(int doc, Explanation freq, BM25Stats stats, NumericDocValues norms) {
+  private Explanation explainScore(int doc, Explanation freq, BM25Stats stats, NumericDocValues norms) throws IOException {
     Explanation boostExpl = Explanation.match(stats.boost, "boost");
     List<Explanation> subs = new ArrayList<>();
     if (boostExpl.getValue() != 1.0f)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
index 68828fc..153fd7b 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
@@ -75,7 +75,7 @@ public class MultiSimilarity extends Similarity {
     }
     
     @Override
-    public float score(int doc, float freq) {
+    public float score(int doc, float freq) throws IOException {
       float sum = 0.0f;
       for (SimScorer subScorer : subScorers) {
         sum += subScorer.score(doc, freq);
@@ -84,7 +84,7 @@ public class MultiSimilarity extends Similarity {
     }
 
     @Override
-    public Explanation explain(int doc, Explanation freq) {
+    public Explanation explain(int doc, Explanation freq) throws IOException {
       List<Explanation> subs = new ArrayList<>();
       for (SimScorer subScorer : subScorers) {
         subs.add(subScorer.explain(doc, freq));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
index 6a6e93b..7f0f27c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
@@ -161,7 +161,7 @@ public abstract class Similarity {
      * @param freq sloppy term frequency
      * @return document's score
      */
-    public abstract float score(int doc, float freq);
+    public abstract float score(int doc, float freq) throws IOException;
 
     /** Computes the amount of a sloppy phrase match, based on an edit distance. */
     public abstract float computeSlopFactor(int distance);
@@ -175,7 +175,7 @@ public abstract class Similarity {
      * @param freq Explanation of how the sloppy term frequency was computed
      * @return document's score
      */
-    public Explanation explain(int doc, Explanation freq) {
+    public Explanation explain(int doc, Explanation freq) throws IOException {
       return Explanation.match(
           score(doc, freq.getValue()),
           "score(doc=" + doc + ",freq=" + freq.getValue() +"), with freq of:",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
index c2c117d..ed837c4 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
@@ -274,17 +274,31 @@ public abstract class SimilarityBase extends Similarity {
       this.stats = stats;
       this.norms = norms;
     }
+
+    private float getNormValue(int doc) throws IOException {
+      if (norms == null) {
+        return 1F;
+      }
+      int normsDocID = norms.docID();
+      if (normsDocID < doc) {
+        normsDocID = norms.advance(doc);
+      }
+      if (normsDocID == doc) {
+        return decodeNormValue((byte) norms.longValue());
+      } else {
+        return decodeNormValue((byte) 0);
+      }
+    }
     
     @Override
-    public float score(int doc, float freq) {
+    public float score(int doc, float freq) throws IOException {
       // We have to supply something in case norms are omitted
-      return SimilarityBase.this.score(stats, freq,
-          norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
+      return SimilarityBase.this.score(stats, freq, getNormValue(doc));
     }
+
     @Override
-    public Explanation explain(int doc, Explanation freq) {
-      return SimilarityBase.this.explain(stats, doc, freq,
-          norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
+    public Explanation explain(int doc, Explanation freq) throws IOException {
+      return SimilarityBase.this.explain(stats, doc, freq, getNormValue(doc));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
index ff708b4..6cd87b5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
+++ b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
@@ -592,10 +592,24 @@ public abstract class TFIDFSimilarity extends Similarity {
     }
     
     @Override
-    public float score(int doc, float freq) {
+    public float score(int doc, float freq) throws IOException {
       final float raw = tf(freq) * weightValue; // compute tf(f)*weight
-      
-      return norms == null ? raw : raw * decodeNormValue(norms.get(doc));  // normalize for field
+
+      if (norms == null) {
+        return raw;
+      } else {
+        long normValue;
+        int normsDocID = norms.docID();
+        if (normsDocID < doc) {
+          normsDocID = norms.advance(doc);
+        }
+        if (normsDocID == doc) {
+          normValue = norms.longValue();
+        } else {
+          normValue = 0;
+        }
+        return raw * decodeNormValue(normValue);  // normalize for field
+      }
     }
     
     @Override
@@ -609,7 +623,7 @@ public abstract class TFIDFSimilarity extends Similarity {
     }
 
     @Override
-    public Explanation explain(int doc, Explanation freq) {
+    public Explanation explain(int doc, Explanation freq) throws IOException {
       return explainScore(doc, freq, stats, norms);
     }
   }
@@ -632,10 +646,17 @@ public abstract class TFIDFSimilarity extends Similarity {
     }
   }  
 
-  private Explanation explainField(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) {
+  private Explanation explainField(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) throws IOException {
     Explanation tfExplanation = Explanation.match(tf(freq.getValue()), "tf(freq="+freq.getValue()+"), with freq of:", freq);
+    float norm;
+    if (norms != null && norms.advance(doc) == doc) {
+      norm = decodeNormValue(norms.longValue());
+    } else {
+      norm = 1f;
+    }
+    
     Explanation fieldNormExpl = Explanation.match(
-        norms != null ? decodeNormValue(norms.get(doc)) : 1.0f,
+        norm,
         "fieldNorm(doc=" + doc + ")");
 
     return Explanation.match(
@@ -644,7 +665,7 @@ public abstract class TFIDFSimilarity extends Similarity {
         tfExplanation, stats.idf, fieldNormExpl);
   }
 
-  private Explanation explainScore(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) {
+  private Explanation explainScore(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) throws IOException {
     Explanation queryExpl = Explanation.match(stats.boost, "boost");
     Explanation fieldExpl = explainField(doc, freq, stats, norms);
     if (stats.boost == 1f) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/util/Bits.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Bits.java b/lucene/core/src/java/org/apache/lucene/util/Bits.java
index cfd3918..101122e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Bits.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Bits.java
@@ -43,7 +43,7 @@ public interface Bits {
   public static class MatchAllBits implements Bits {
     final int len;
     
-    public MatchAllBits( int len ) {
+    public MatchAllBits(int len) {
       this.len = len;
     }
 
@@ -64,7 +64,7 @@ public interface Bits {
   public static class MatchNoBits implements Bits {
     final int len;
     
-    public MatchNoBits( int len ) {
+    public MatchNoBits(int len) {
       this.len = len;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/util/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/LongValues.java b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
index e4b8a35..aa3b7fa 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LongValues.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
@@ -17,15 +17,20 @@
 package org.apache.lucene.util;
 
 
+import org.apache.lucene.index.LegacyNumericDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.util.packed.PackedInts;
 
 /** Abstraction over an array of longs.
  *  This class extends NumericDocValues so that we don't need to add another
  *  level of abstraction every time we want eg. to use the {@link PackedInts}
- *  utility classes to represent a {@link NumericDocValues} instance.
- *  @lucene.internal */
-public abstract class LongValues extends NumericDocValues {
+ *  utility classes to represent a {@link LegacyNumericDocValues} instance.
+ *  @lucene.internal
+ *
+ *  @deprecated Switch to {@link NumericDocValues} instead. */
+ @Deprecated
+// TODO: cutover to iterator once codecs have all cutover?
+public abstract class LongValues extends LegacyNumericDocValues {
 
   /** An instance that returns the provided value. */
   public static final LongValues IDENTITY = new LongValues() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java b/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
index e028c2c..054f3a3 100644
--- a/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
+++ b/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.LegacyNumericDocValues;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
@@ -449,7 +449,7 @@ public class PackedInts {
    * A read-only random access array of positive integers.
    * @lucene.internal
    */
-  public static abstract class Reader extends NumericDocValues implements Accountable {
+  public static abstract class Reader extends LegacyNumericDocValues implements Accountable {
 
     /**
      * Bulk get: read at least one and at most <code>len</code> longs starting

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
index 58d980e..f798148 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene54/TestLucene54DocValuesFormat.java
@@ -66,7 +66,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.store.RAMInputStream;
 import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.LongValues;
@@ -206,19 +205,14 @@ public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatT
     for (LeafReaderContext context : indexReader.leaves()) {
       final LeafReader reader = context.reader();
       final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
-      final Bits numericBits = DocValues.getDocsWithField(reader, "numeric");
 
       final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
-      final Bits sortedBits = DocValues.getDocsWithField(reader, "sorted");
 
       final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
-      final Bits binaryBits = DocValues.getDocsWithField(reader, "binary");
 
       final SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(reader, "sorted_numeric");
-      final Bits sortedNumericBits = DocValues.getDocsWithField(reader, "sorted_numeric");
 
       final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
-      final Bits sortedSetBits = DocValues.getDocsWithField(reader, "sorted_set");
 
       for (int i = 0; i < reader.maxDoc(); ++i) {
         final Document doc = reader.document(i);
@@ -226,49 +220,43 @@ public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatT
         final Long value = valueField == null ? null : valueField.numericValue().longValue();
 
         if (value == null) {
-          assertEquals(0, numeric.get(i));
-          assertEquals(-1, sorted.getOrd(i));
-          assertEquals(new BytesRef(), binary.get(i));
-
-          assertFalse(numericBits.get(i));
-          assertFalse(sortedBits.get(i));
-          assertFalse(binaryBits.get(i));
+          assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
         } else {
-          assertEquals(value.longValue(), numeric.get(i));
-          assertTrue(sorted.getOrd(i) >= 0);
-          assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.getOrd(i)));
-          assertEquals(new BytesRef(Long.toString(value)), binary.get(i));
-
-          assertTrue(numericBits.get(i));
-          assertTrue(sortedBits.get(i));
-          assertTrue(binaryBits.get(i));
+          assertEquals(i, numeric.nextDoc());
+          assertEquals(i, binary.nextDoc());
+          assertEquals(i, sorted.nextDoc());
+          assertEquals(value.longValue(), numeric.longValue());
+          assertTrue(sorted.ordValue() >= 0);
+          assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
+          assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
         }
 
         final IndexableField[] valuesFields = doc.getFields("values");
-        final Set<Long> valueSet = new HashSet<>();
-        for (IndexableField sf : valuesFields) {
-          valueSet.add(sf.numericValue().longValue());
-        }
+        if (valuesFields.length == 0) {
+          assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
+        } else {
+          final Set<Long> valueSet = new HashSet<>();
+          for (IndexableField sf : valuesFields) {
+            valueSet.add(sf.numericValue().longValue());
+          }
 
-        sortedNumeric.setDocument(i);
-        assertEquals(valuesFields.length, sortedNumeric.count());
-        for (int j = 0; j < sortedNumeric.count(); ++j) {
-          assertTrue(valueSet.contains(sortedNumeric.valueAt(j)));
-        }
-        sortedSet.setDocument(i);
-        int sortedSetCount = 0;
-        while (true) {
-          long ord = sortedSet.nextOrd();
-          if (ord == SortedSetDocValues.NO_MORE_ORDS) {
-            break;
+          assertEquals(i, sortedNumeric.nextDoc());
+          assertEquals(valuesFields.length, sortedNumeric.docValueCount());
+          for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
+            assertTrue(valueSet.contains(sortedNumeric.nextValue()));
+          }
+          assertEquals(i, sortedSet.nextDoc());
+          int sortedSetCount = 0;
+          while (true) {
+            long ord = sortedSet.nextOrd();
+            if (ord == SortedSetDocValues.NO_MORE_ORDS) {
+              break;
+            }
+            assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
+            sortedSetCount++;
           }
-          assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
-          sortedSetCount++;
+          assertEquals(valueSet.size(), sortedSetCount);
         }
-        assertEquals(valueSet.size(), sortedSetCount);
-
-        assertEquals(!valueSet.isEmpty(), sortedNumericBits.get(i));
-        assertEquals(!valueSet.isEmpty(), sortedSetBits.get(i));
       }
     }
 
@@ -546,7 +534,7 @@ public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatT
       RAMInputStream in = new RAMInputStream("", buffer);
       BytesRefBuilder b = new BytesRefBuilder();
       for (int i = 0; i < maxDoc; ++i) {
-        values.setDocument(i);
+        assertEquals(i, values.nextDoc());
         final int numValues = in.readVInt();
 
         for (int j = 0; j < numValues; ++j) {
@@ -595,10 +583,10 @@ public class TestLucene54DocValuesFormat extends BaseCompressingDocValuesFormatT
       assertNotNull(values);
       RAMInputStream in = new RAMInputStream("", buffer);
       for (int i = 0; i < maxDoc; ++i) {
-        values.setDocument(i);
-        assertEquals(2, values.count());
-        assertEquals(in.readVLong(), values.valueAt(0));
-        assertEquals(in.readVLong(), values.valueAt(1));
+        assertEquals(i, values.nextDoc());
+        assertEquals(2, values.docValueCount());
+        assertEquals(in.readVLong(), values.nextValue());
+        assertEquals(in.readVLong(), values.nextValue());
       }
       r.close();
       dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
index 1ebfb69..5777dad 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
@@ -110,13 +110,17 @@ public class TestPerFieldDocValuesFormat extends BaseDocValuesFormatTestCase {
     assertEquals(1, hits.totalHits);
     // Iterate through the results:
     for (int i = 0; i < hits.scoreDocs.length; i++) {
-      Document hitDoc = isearcher.doc(hits.scoreDocs[i].doc);
+      int hitDocID = hits.scoreDocs[i].doc;
+      Document hitDoc = isearcher.doc(hitDocID);
       assertEquals(text, hitDoc.get("fieldname"));
       assert ireader.leaves().size() == 1;
       NumericDocValues dv = ireader.leaves().get(0).reader().getNumericDocValues("dv1");
-      assertEquals(5, dv.get(hits.scoreDocs[i].doc));
+      assertEquals(hitDocID, dv.advance(hitDocID));
+      assertEquals(5, dv.longValue());
+      
       BinaryDocValues dv2 = ireader.leaves().get(0).reader().getBinaryDocValues("dv2");
-      final BytesRef term = dv2.get(hits.scoreDocs[i].doc);
+      assertEquals(hitDocID, dv2.advance(hitDocID));
+      final BytesRef term = dv2.binaryValue();
       assertEquals(new BytesRef("hello world"), term);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java b/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java
index 9f273df..4662480 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BBinaryDocValues.java
@@ -91,7 +91,8 @@ public class Test2BBinaryDocValues extends LuceneTestCase {
         bytes[1] = (byte)(expectedValue >> 16);
         bytes[2] = (byte)(expectedValue >> 8);
         bytes[3] = (byte) expectedValue;
-        final BytesRef term = dv.get(i);
+        assertEquals(i, dv.nextDoc());
+        final BytesRef term = dv.binaryValue();
         assertEquals(data, term);
         expectedValue++;
       }
@@ -148,7 +149,8 @@ public class Test2BBinaryDocValues extends LuceneTestCase {
       LeafReader reader = context.reader();
       BinaryDocValues dv = reader.getBinaryDocValues("dv");
       for (int i = 0; i < reader.maxDoc(); i++) {
-        final BytesRef term = dv.get(i);
+        assertEquals(i, dv.nextDoc());
+        final BytesRef term = dv.binaryValue();
         input.reset(term.bytes, term.offset, term.length);
         assertEquals(expectedValue % 65535, input.readVInt());
         assertTrue(input.eof());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java b/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java
index 2460073..046ac6f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BNumericDocValues.java
@@ -80,7 +80,8 @@ public class Test2BNumericDocValues extends LuceneTestCase {
       LeafReader reader = context.reader();
       NumericDocValues dv = reader.getNumericDocValues("dv");
       for (int i = 0; i < reader.maxDoc(); i++) {
-        assertEquals(expectedValue, dv.get(i));
+        assertEquals(i, dv.nextDoc());
+        assertEquals(expectedValue, dv.longValue());
         expectedValue++;
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
index 56bf4b5..988e3f7 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
@@ -82,11 +82,12 @@ public class Test2BSortedDocValuesFixedSorted extends LuceneTestCase {
     int expectedValue = 0;
     for (LeafReaderContext context : r.leaves()) {
       LeafReader reader = context.reader();
-      BinaryDocValues dv = reader.getSortedDocValues("dv");
+      BinaryDocValues dv = DocValues.getBinary(reader, "dv");
       for (int i = 0; i < reader.maxDoc(); i++) {
+        assertEquals(i, dv.nextDoc());
         bytes[0] = (byte)(expectedValue >> 8);
         bytes[1] = (byte) expectedValue;
-        final BytesRef term = dv.get(i);
+        final BytesRef term = dv.binaryValue();
         assertEquals(data, term);
         expectedValue++;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
index 70ed6bf..7448738 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
@@ -85,14 +85,15 @@ public class Test2BSortedDocValuesOrds extends LuceneTestCase {
     for (LeafReaderContext context : r.leaves()) {
       LeafReader reader = context.reader();
       BytesRef scratch = new BytesRef();
-      BinaryDocValues dv = reader.getSortedDocValues("dv");
+      BinaryDocValues dv = DocValues.getBinary(reader, "dv");
       for (int i = 0; i < reader.maxDoc(); i++) {
+        assertEquals(i, dv.nextDoc());
         bytes[0] = (byte) (counter >> 24);
         bytes[1] = (byte) (counter >> 16);
         bytes[2] = (byte) (counter >> 8);
         bytes[3] = (byte) counter;
         counter++;
-        final BytesRef term = dv.get(i);
+        final BytesRef term = dv.binaryValue();
         assertEquals(data, term);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
----------------------------------------------------------------------
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 c6cf1c3..f552ae9 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
@@ -46,12 +46,15 @@ import org.junit.Test;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 
 public class TestBinaryDocValuesUpdates extends LuceneTestCase {
 
-  static long getValue(BinaryDocValues bdv, int idx) {
-    BytesRef term = bdv.get(idx);
-    idx = term.offset;
+  static long getValue(BinaryDocValues bdv) {
+    BytesRef term = bdv.binaryValue();
+    int idx = term.offset;
+    assert term.length > 0;
     byte b = term.bytes[idx++];
     long value = b & 0x7FL;
     for (int shift = 7; (b & 0x80L) != 0; shift += 7) {
@@ -129,8 +132,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     assertEquals(1, reader.leaves().size());
     LeafReader r = reader.leaves().get(0).reader();
     BinaryDocValues bdv = r.getBinaryDocValues("val");
-    assertEquals(2, getValue(bdv, 0));
-    assertEquals(2, getValue(bdv, 1));
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(2, getValue(bdv));
+    assertEquals(1, bdv.nextDoc());
+    assertEquals(2, getValue(bdv));
     reader.close();
     
     dir.close();
@@ -173,8 +178,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
       BinaryDocValues bdv = r.getBinaryDocValues("val");
       assertNotNull(bdv);
       for (int i = 0; i < r.maxDoc(); i++) {
+        assertEquals(i, bdv.nextDoc());
         long expected = expectedValues[i + context.docBase];
-        long actual = getValue(bdv, i);
+        long actual = getValue(bdv);
         assertEquals(expected, actual);
       }
     }
@@ -212,8 +218,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
 
     BinaryDocValues bdv1 = reader1.leaves().get(0).reader().getBinaryDocValues("val");
     BinaryDocValues bdv2 = reader2.leaves().get(0).reader().getBinaryDocValues("val");
-    assertEquals(1, getValue(bdv1, 0));
-    assertEquals(10, getValue(bdv2, 0));
+    assertEquals(0, bdv1.nextDoc());
+    assertEquals(1, getValue(bdv1));
+    assertEquals(0, bdv2.nextDoc());
+    assertEquals(10, getValue(bdv2));
 
     writer.close();
     IOUtils.close(reader1, reader2, dir);
@@ -260,7 +268,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     long[] expectedValues = new long[] { 1, 2, 3, 17, 5, 17};
     BinaryDocValues bdv = MultiDocValues.getBinaryValues(reader, "val");
     for (int i = 0; i < expectedValues.length; i++) {
-      assertEquals(expectedValues[i], getValue(bdv, i));
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(expectedValues[i], getValue(bdv));
     }
     
     reader.close();
@@ -295,7 +304,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     
     LeafReader r = reader.leaves().get(0).reader();
     assertFalse(r.getLiveDocs().get(0));
-    assertEquals(17, getValue(r.getBinaryDocValues("val"), 1));
+    BinaryDocValues bdv = r.getBinaryDocValues("val");
+    assertEquals(1, bdv.advance(1));
+    assertEquals(17, getValue(bdv));
     
     reader.close();
     dir.close();
@@ -330,14 +341,18 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     SortedDocValues sdv = r.getSortedDocValues("sdv");
     SortedSetDocValues ssdv = r.getSortedSetDocValues("ssdv");
     for (int i = 0; i < r.maxDoc(); i++) {
-      assertEquals(i, ndv.get(i));
-      assertEquals(17, getValue(bdv, i));
-      BytesRef term = sdv.get(i);
+      assertEquals(i, ndv.nextDoc());
+      assertEquals(i, ndv.longValue());
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(17, getValue(bdv));
+      assertEquals(i, sdv.nextDoc());
+      BytesRef term = sdv.binaryValue();
       assertEquals(new BytesRef(Integer.toString(i)), term);
-      ssdv.setDocument(i);
+      assertEquals(i, ssdv.nextDoc());
       long ord = ssdv.nextOrd();
       term = ssdv.lookupOrd(ord);
       assertEquals(i, Integer.parseInt(term.utf8ToString()));
+      // For the i=0 case, we added the same value twice, which was dedup'd by IndexWriter so it has only one value:
       if (i != 0) {
         ord = ssdv.nextOrd();
         term = ssdv.lookupOrd(ord);
@@ -375,8 +390,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     BinaryDocValues bdv1 = r.getBinaryDocValues("bdv1");
     BinaryDocValues bdv2 = r.getBinaryDocValues("bdv2");
     for (int i = 0; i < r.maxDoc(); i++) {
-      assertEquals(17, getValue(bdv1, i));
-      assertEquals(i, getValue(bdv2, i));
+      assertEquals(i, bdv1.nextDoc());
+      assertEquals(17, getValue(bdv1));
+      assertEquals(i, bdv2.nextDoc());
+      assertEquals(i, getValue(bdv2));
     }
     
     reader.close();
@@ -406,7 +423,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     LeafReader r = reader.leaves().get(0).reader();
     BinaryDocValues bdv = r.getBinaryDocValues("bdv");
     for (int i = 0; i < r.maxDoc(); i++) {
-      assertEquals(17, getValue(bdv, i));
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(17, getValue(bdv));
     }
     
     reader.close();
@@ -468,8 +486,10 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     BinaryDocValues bdv = MultiDocValues.getBinaryValues(reader, "bdv");
     SortedDocValues sdv = MultiDocValues.getSortedValues(reader, "sorted");
     for (int i = 0; i < reader.maxDoc(); i++) {
-      assertEquals(17, getValue(bdv, i));
-      BytesRef term = sdv.get(i);
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(17, getValue(bdv));
+      assertEquals(i, sdv.nextDoc());
+      BytesRef term = sdv.binaryValue();
       assertEquals(new BytesRef("value"), term);
     }
     
@@ -496,7 +516,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     final DirectoryReader reader = DirectoryReader.open(dir);
     BinaryDocValues bdv = MultiDocValues.getBinaryValues(reader, "bdv");
     for (int i = 0; i < reader.maxDoc(); i++) {
-      assertEquals(3, getValue(bdv, i));
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(3, getValue(bdv));
     }
     reader.close();
     dir.close();
@@ -507,7 +528,7 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     Random random = random();
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random));
     IndexWriter writer = new IndexWriter(dir, conf);
-    
+
     int docid = 0;
     int numRounds = atLeast(10);
     for (int rnd = 0; rnd < numRounds; rnd++) {
@@ -564,7 +585,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
       BinaryDocValues bdv = r.getBinaryDocValues("bdv");
       assertNotNull(bdv);
       for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(value, getValue(bdv, i));
+        assertEquals(i, bdv.nextDoc());
+        assertEquals(value, getValue(bdv));
       }
       reader.close();
     }
@@ -594,7 +616,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     final DirectoryReader reader = DirectoryReader.open(dir);
     BinaryDocValues bdv = MultiDocValues.getBinaryValues(reader, "bdv");
     for (int i = 0; i < reader.maxDoc(); i++) {
-      assertEquals(3, getValue(bdv, i));
+      assertEquals(i, bdv.nextDoc());
+      assertEquals(3, getValue(bdv));
     }
     reader.close();
     dir.close();
@@ -672,14 +695,12 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
         for (int field = 0; field < fieldValues.length; field++) {
           String f = "f" + field;
           BinaryDocValues bdv = r.getBinaryDocValues(f);
-          Bits docsWithField = r.getDocsWithField(f);
           assertNotNull(bdv);
           int maxDoc = r.maxDoc();
           for (int doc = 0; doc < maxDoc; doc++) {
             if (liveDocs == null || liveDocs.get(doc)) {
-//              System.out.println("doc=" + (doc + context.docBase) + " f='" + f + "' vslue=" + getValue(bdv, doc, scratch));
-              assertTrue(docsWithField.get(doc));
-              assertEquals("invalid value for doc=" + doc + ", field=" + f + ", reader=" + r, fieldValues[field], getValue(bdv, doc));
+              assertEquals(doc, bdv.advance(doc));
+              assertEquals("invalid value for doc=" + doc + ", field=" + f + ", reader=" + r, fieldValues[field], getValue(bdv));
             }
           }
         }
@@ -732,13 +753,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     for (LeafReaderContext context : reader.leaves()) {
       LeafReader r = context.reader();
       BinaryDocValues bdv = r.getBinaryDocValues("bdv");
-      Bits docsWithField = r.getDocsWithField("bdv");
-      assertNotNull(docsWithField);
-      assertTrue(docsWithField.get(0));
-      assertEquals(5L, getValue(bdv, 0));
-      assertFalse(docsWithField.get(1));
-      BytesRef term = bdv.get(1);
-      assertEquals(0, term.length);
+      assertEquals(0, bdv.nextDoc());
+      assertEquals(5L, getValue(bdv));
+      assertEquals(NO_MORE_DOCS, bdv.nextDoc());
     }
     reader.close();
 
@@ -778,7 +795,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
       LeafReader r = context.reader();
       BinaryDocValues bdv = r.getBinaryDocValues("bdv");
       for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(5L, getValue(bdv, i));
+        assertEquals(i, bdv.nextDoc());
+        assertEquals(5L, getValue(bdv));
       }
     }
     reader.close();
@@ -803,7 +821,8 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     
     DirectoryReader r = DirectoryReader.open(dir);
     BinaryDocValues bdv = r.leaves().get(0).reader().getBinaryDocValues("f");
-    assertEquals(17, getValue(bdv, 0));
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(17, getValue(bdv));
     r.close();
     
     dir.close();
@@ -920,14 +939,12 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
       for (int i = 0; i < numFields; i++) {
         BinaryDocValues bdv = r.getBinaryDocValues("f" + i);
         BinaryDocValues control = r.getBinaryDocValues("cf" + i);
-        Bits docsWithBdv = r.getDocsWithField("f" + i);
-        Bits docsWithControl = r.getDocsWithField("cf" + i);
         Bits liveDocs = r.getLiveDocs();
         for (int j = 0; j < r.maxDoc(); j++) {
           if (liveDocs == null || liveDocs.get(j)) {
-            assertTrue(docsWithBdv.get(j));
-            assertTrue(docsWithControl.get(j));
-            assertEquals(getValue(control, j), getValue(bdv, j) * 2);
+            assertEquals(j, bdv.advance(j));
+            assertEquals(j, control.advance(j));
+            assertEquals(getValue(control), getValue(bdv) * 2);
           }
         }
       }
@@ -965,7 +982,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
         BinaryDocValues fbdv = r.getBinaryDocValues("f");
         BinaryDocValues cfbdv = r.getBinaryDocValues("cf");
         for (int j = 0; j < r.maxDoc(); j++) {
-          assertEquals(getValue(cfbdv, j), getValue(fbdv, j) * 2);
+          assertEquals(j, fbdv.nextDoc());
+          assertEquals(j, cfbdv.nextDoc());
+          assertEquals(getValue(cfbdv), getValue(fbdv) * 2);
         }
       }
       reader.close();
@@ -1013,10 +1032,14 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     DirectoryReader reader = DirectoryReader.open(dir);
     BinaryDocValues f1 = MultiDocValues.getBinaryValues(reader, "f1");
     BinaryDocValues f2 = MultiDocValues.getBinaryValues(reader, "f2");
-    assertEquals(12L, getValue(f1, 0));
-    assertEquals(13L, getValue(f2, 0));
-    assertEquals(17L, getValue(f1, 1));
-    assertEquals(2L, getValue(f2, 1));
+    assertEquals(0, f1.nextDoc());
+    assertEquals(0, f2.nextDoc());
+    assertEquals(12L, getValue(f1));
+    assertEquals(13L, getValue(f2));
+    assertEquals(1, f1.nextDoc());
+    assertEquals(1, f2.nextDoc());
+    assertEquals(17L, getValue(f1));
+    assertEquals(2L, getValue(f2));
     reader.close();
     dir.close();
   }
@@ -1070,7 +1093,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
       BinaryDocValues bdv = r.getBinaryDocValues("bdv");
       BinaryDocValues control = r.getBinaryDocValues("control");
       for (int i = 0; i < r.maxDoc(); i++) {
-        assertEquals(getValue(bdv, i)*2, getValue(control, i));
+        assertEquals(i, bdv.nextDoc());
+        assertEquals(i, control.nextDoc());
+        assertEquals(getValue(bdv)*2, getValue(control));
       }
     }
     reader.close();
@@ -1167,7 +1192,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
         BinaryDocValues f = r.getBinaryDocValues("f" + i);
         BinaryDocValues cf = r.getBinaryDocValues("cf" + i);
         for (int j = 0; j < r.maxDoc(); j++) {
-          assertEquals("reader=" + r + ", field=f" + i + ", doc=" + j, getValue(cf, j), getValue(f, j) * 2);
+          assertEquals(j, f.nextDoc());
+          assertEquals(j, cf.nextDoc());
+          assertEquals("reader=" + r + ", field=f" + i + ", doc=" + j, getValue(cf), getValue(f) * 2);
         }
       }
     }
@@ -1195,8 +1222,12 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     writer.close();
     
     DirectoryReader reader = DirectoryReader.open(dir);
-    assertEquals(4, getValue(reader.leaves().get(0).reader().getBinaryDocValues("f1"), 0));
-    assertEquals(3, getValue(reader.leaves().get(0).reader().getBinaryDocValues("f2"), 0));
+    BinaryDocValues bdv = reader.leaves().get(0).reader().getBinaryDocValues("f1");
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(4, getValue(bdv));
+    bdv = reader.leaves().get(0).reader().getBinaryDocValues("f2");
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(3, getValue(bdv));
     reader.close();
     
     dir.close();
@@ -1220,7 +1251,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     
     DirectoryReader reader = DirectoryReader.open(dir);
     assertEquals(1, reader.leaves().size());
-    assertEquals(2L, getValue(reader.leaves().get(0).reader().getBinaryDocValues("f1"), 0));
+    BinaryDocValues bdv = reader.leaves().get(0).reader().getBinaryDocValues("f1");
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(2L, getValue(bdv));
     reader.close();
     
     dir.close();
@@ -1242,7 +1275,9 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
     
     DirectoryReader reader = DirectoryReader.open(dir);
     assertEquals(1, reader.leaves().size());
-    assertEquals(1L, getValue(reader.leaves().get(0).reader().getBinaryDocValues("f1"), 0));
+    BinaryDocValues bdv = reader.leaves().get(0).reader().getBinaryDocValues("f1");
+    assertEquals(0, bdv.nextDoc());
+    assertEquals(1L, getValue(bdv));
     reader.close();
     
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
index ba3414b..65310cf 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java
@@ -72,7 +72,8 @@ public class TestCustomNorms extends LuceneTestCase {
     for (int i = 0; i < open.maxDoc(); i++) {
       Document document = open.document(i);
       float expected = Float.parseFloat(document.get(floatTestField));
-      assertEquals(expected, Float.intBitsToFloat((int)norms.get(i)), 0.0f);
+      assertEquals(i, norms.nextDoc());
+      assertEquals(expected, Float.intBitsToFloat((int)norms.longValue()), 0.0f);
     }
     open.close();
     dir.close();