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:10 UTC

[06/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/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
index 44f3f8c..129b9348 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
@@ -183,13 +183,21 @@ public final class DocValuesRangeQuery extends Query {
 
             @Override
             public boolean get(int doc) {
-              values.setDocument(doc);
-              final int count = values.count();
-              for (int i = 0; i < count; ++i) {
-                final long value = values.valueAt(i);
-                if (value >= min && value <= max) {
-                  return true;
+              try {
+                if (doc > values.docID()) {
+                  values.advance(doc);
                 }
+                if (doc == values.docID()) {
+                  final int count = values.docValueCount();
+                  for (int i = 0; i < count; ++i) {
+                    final long value = values.nextValue();
+                    if (value >= min && value <= max) {
+                      return true;
+                    }
+                  }
+                }
+              } catch (IOException ioe) {
+                throw new RuntimeException(ioe);
               }
               return false;
             }
@@ -241,11 +249,19 @@ public final class DocValuesRangeQuery extends Query {
 
             @Override
             public boolean get(int doc) {
-              values.setDocument(doc);
-              for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
-                if (ord >= minOrd && ord <= maxOrd) {
-                  return true;
+              try {
+                if (doc > values.docID()) {
+                  values.advance(doc);
+                }
+                if (doc == values.docID()) {
+                  for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+                    if (ord >= minOrd && ord <= maxOrd) {
+                      return true;
+                    }
+                  }
                 }
+              } catch (IOException ioe) {
+                throw new RuntimeException(ioe);
               }
               return false;
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index 4be4b18..92037a8 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -165,11 +165,19 @@ public class DocValuesTermsQuery extends Query {
 
           @Override
           public boolean get(int doc) {
-            values.setDocument(doc);
-            for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
-              if (bits.get(ord)) {
-                return true;
+            try {
+              if (doc > values.docID()) {
+                values.advance(doc);
               }
+              if (doc == values.docID()) {
+                for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+                  if (bits.get(ord)) {
+                    return true;
+                  }
+                }
+              }
+            } catch (IOException ioe) {
+              throw new RuntimeException(ioe);
             }
             return false;
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
index 9c5cf7f..0a1755c 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
@@ -360,7 +360,7 @@ class TermAutomatonScorer extends Scorer {
   }
 
   @Override
-  public float score() {
+  public float score() throws IOException {
     // TODO: we could probably do better here, e.g. look @ freqs of actual terms involved in this doc and score differently
     return docScorer.score(docID, freq);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
index 9d29330..ff61ff6 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -203,7 +203,6 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
 
     int dimensions = ranges[0][0].numDimensions();
     int iters = atLeast(25);
-    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
     Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
     int maxDoc = s.getIndexReader().maxDoc();
 
@@ -250,8 +249,10 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
         public boolean needsScores() { return false; }
       });
 
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
       for (int docID=0; docID<maxDoc; ++docID) {
-        int id = (int) docIDToID.get(docID);
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
         boolean expected;
         if (liveDocs != null && liveDocs.get(docID) == false) {
           // document is deleted

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
index 1d8b4b0..b1424e0 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
@@ -67,22 +67,23 @@ public abstract class BBoxSimilarityValueSource extends ValueSource {
 
     return new DoubleDocValues(this) {
       @Override
-      public double doubleVal(int doc) {
+      public double doubleVal(int doc) throws IOException {
         //? limit to Rect or call getBoundingBox()? latter would encourage bad practice
         final Rectangle rect = (Rectangle) shapeValues.objectVal(doc);
         return rect==null ? 0 : score(rect, null);
       }
 
       @Override
-      public boolean exists(int doc) {
+      public boolean exists(int doc) throws IOException {
         return shapeValues.exists(doc);
       }
 
       @Override
-      public Explanation explain(int doc) {
+      public Explanation explain(int doc) throws IOException {
         final Rectangle rect = (Rectangle) shapeValues.objectVal(doc);
-        if (rect == null)
+        if (rect == null) {
           return Explanation.noMatch("no rect");
+        }
         AtomicReference<Explanation> explanation = new AtomicReference<>();
         score(rect, explanation);
         return explanation.get();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
index 2bfbfd9..ef9f54e 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
@@ -16,18 +16,17 @@
  */
 package org.apache.lucene.spatial.bbox;
 
-import org.locationtech.spatial4j.shape.Rectangle;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.util.Bits;
-
-import java.io.IOException;
-import java.util.Map;
+import org.locationtech.spatial4j.shape.Rectangle;
 
 /**
  * A ValueSource in which the indexed Rectangle is returned from
@@ -56,41 +55,62 @@ class BBoxValueSource extends ValueSource {
     final NumericDocValues maxX = DocValues.getNumeric(reader, strategy.field_maxX);
     final NumericDocValues maxY = DocValues.getNumeric(reader, strategy.field_maxY);
 
-    final Bits validBits = DocValues.getDocsWithField(reader, strategy.field_minX);//could have chosen any field
     //reused
     final Rectangle rect = strategy.getSpatialContext().makeRectangle(0,0,0,0);
 
     return new FunctionValues() {
+      private int lastDocID = -1;
+
+      private double getDocValue(NumericDocValues values, int doc) throws IOException {
+        int curDocID = values.docID();
+        if (doc > curDocID) {
+          curDocID = values.advance(doc);
+        }
+        if (doc == curDocID) {
+          return Double.longBitsToDouble(values.longValue());
+        } else {
+          return 0.0;
+        }
+      }
+
       @Override
-      public Object objectVal(int doc) {
-        if (!validBits.get(doc)) {
+      public Object objectVal(int doc) throws IOException {
+        if (doc < lastDocID) {
+          throw new AssertionError("docs were sent out-of-order: lastDocID=" + lastDocID + " vs doc=" + doc);
+        }
+        lastDocID = doc;
+
+        double minXValue = getDocValue(minX, doc);
+        if (minX.docID() != doc) {
           return null;
         } else {
-          rect.reset(
-              Double.longBitsToDouble(minX.get(doc)), Double.longBitsToDouble(maxX.get(doc)),
-              Double.longBitsToDouble(minY.get(doc)), Double.longBitsToDouble(maxY.get(doc)));
+          double minYValue = getDocValue(minY, doc);
+          double maxXValue = getDocValue(maxX, doc);
+          double maxYValue = getDocValue(maxY, doc);
+          rect.reset(minXValue, maxXValue, minYValue, maxYValue);
           return rect;
         }
       }
 
       @Override
-      public String strVal(int doc) {//TODO support WKT output once Spatial4j does
+      public String strVal(int doc) throws IOException {//TODO support WKT output once Spatial4j does
         Object v = objectVal(doc);
         return v == null ? null : v.toString();
       }
 
       @Override
-      public boolean exists(int doc) {
-        return validBits.get(doc);
+      public boolean exists(int doc) throws IOException {
+        getDocValue(minX, doc);
+        return minX.docID() == doc;
       }
 
       @Override
-      public Explanation explain(int doc) {
+      public Explanation explain(int doc) throws IOException {
         return Explanation.match(Float.NaN, toString(doc));
       }
 
       @Override
-      public String toString(int doc) {
+      public String toString(int doc) throws IOException {
         return description() + '=' + strVal(doc);
       }
     };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
index c2093f2..d9c45f1 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
@@ -24,10 +24,6 @@ import java.io.FilterOutputStream;
 import java.io.IOException;
 import java.util.Map;
 
-import org.locationtech.spatial4j.context.SpatialContext;
-import org.locationtech.spatial4j.io.BinaryCodec;
-import org.locationtech.spatial4j.shape.Point;
-import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.BinaryDocValues;
@@ -47,6 +43,10 @@ import org.apache.lucene.spatial.util.ShapePredicateValueSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.io.BinaryCodec;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 
 /**
@@ -143,7 +143,11 @@ public class SerializedDVStrategy extends SpatialStrategy {
           return new Bits() {
             @Override
             public boolean get(int index) {
-              return predFuncValues.boolVal(index);
+              try {
+                return predFuncValues.boolVal(index);
+              } catch (IOException ioe) {
+                throw new RuntimeException(ioe);
+              }
             }
 
             @Override
@@ -196,21 +200,28 @@ public class SerializedDVStrategy extends SpatialStrategy {
         int bytesRefDoc = -1;
         BytesRefBuilder bytesRef = new BytesRefBuilder();
 
-        boolean fillBytes(int doc) {
+        boolean fillBytes(int doc) throws IOException {
           if (bytesRefDoc != doc) {
-            bytesRef.copyBytes(docValues.get(doc));
+            if (docValues.docID() < doc) {
+              docValues.advance(doc);
+            }
+            if (docValues.docID() == doc) {
+              bytesRef.copyBytes(docValues.binaryValue());
+            } else {
+              bytesRef.clear();
+            }
             bytesRefDoc = doc;
           }
           return bytesRef.length() != 0;
         }
 
         @Override
-        public boolean exists(int doc) {
+        public boolean exists(int doc) throws IOException {
           return fillBytes(doc);
         }
 
         @Override
-        public boolean bytesVal(int doc, BytesRefBuilder target) {
+        public boolean bytesVal(int doc, BytesRefBuilder target) throws IOException {
           target.clear();
           if (fillBytes(doc)) {
             target.copyBytes(bytesRef);
@@ -221,7 +232,7 @@ public class SerializedDVStrategy extends SpatialStrategy {
         }
 
         @Override
-        public Object objectVal(int docId) {
+        public Object objectVal(int docId) throws IOException {
           if (!fillBytes(docId))
             return null;
           DataInputStream dataInput = new DataInputStream(
@@ -234,12 +245,12 @@ public class SerializedDVStrategy extends SpatialStrategy {
         }
 
         @Override
-        public Explanation explain(int doc) {
+        public Explanation explain(int doc) throws IOException {
           return Explanation.match(Float.NaN, toString(doc));
         }
 
         @Override
-        public String toString(int doc) {
+        public String toString(int doc) throws IOException {
           return description() + "=" + objectVal(doc);//TODO truncate?
         }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
index 73d25ca..d43d4e8 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
@@ -52,7 +52,7 @@ public class CachingDoubleValueSource extends ValueSource {
     return new FunctionValues() {
 
       @Override
-      public double doubleVal(int doc) {
+      public double doubleVal(int doc) throws IOException {
         Integer key = Integer.valueOf( base+doc );
         Double v = cache.get( key );
         if( v == null ) {
@@ -63,12 +63,12 @@ public class CachingDoubleValueSource extends ValueSource {
       }
 
       @Override
-      public float floatVal(int doc) {
+      public float floatVal(int doc) throws IOException {
         return (float)doubleVal(doc);
       }
 
       @Override
-      public String toString(int doc) {
+      public String toString(int doc) throws IOException {
         return doubleVal(doc)+"";
       }
     };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
index 7be2433..6993adb 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
@@ -76,7 +76,7 @@ public class DistanceToShapeValueSource extends ValueSource {
 
     return new DoubleDocValues(this) {
       @Override
-      public double doubleVal(int doc) {
+      public double doubleVal(int doc) throws IOException {
         Shape shape = (Shape) shapeValues.objectVal(doc);
         if (shape == null || shape.isEmpty())
           return nullValue;
@@ -85,7 +85,7 @@ public class DistanceToShapeValueSource extends ValueSource {
       }
 
       @Override
-      public Explanation explain(int doc) {
+      public Explanation explain(int doc) throws IOException {
         Explanation exp = super.explain(doc);
         List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
         details.add(shapeValues.explain(doc));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
index 257dc67..9c2c0e3 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
@@ -69,7 +69,7 @@ public class ShapeAreaValueSource extends ValueSource {
 
     return new DoubleDocValues(this) {
       @Override
-      public double doubleVal(int doc) {
+      public double doubleVal(int doc) throws IOException {
         Shape shape = (Shape) shapeValues.objectVal(doc);
         if (shape == null || shape.isEmpty())
           return 0;//or NaN?
@@ -79,12 +79,12 @@ public class ShapeAreaValueSource extends ValueSource {
       }
 
       @Override
-      public boolean exists(int doc) {
+      public boolean exists(int doc) throws IOException {
         return shapeValues.exists(doc);
       }
 
       @Override
-      public Explanation explain(int doc) {
+      public Explanation explain(int doc) throws IOException {
         Explanation exp = super.explain(doc);
         List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
         details.add(shapeValues.explain(doc));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
index 08c1e43..d0dc51f 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
@@ -72,7 +72,7 @@ public class ShapePredicateValueSource extends ValueSource {
 
     return new BoolDocValues(this) {
       @Override
-      public boolean boolVal(int doc) {
+      public boolean boolVal(int doc) throws IOException {
         Shape indexedShape = (Shape) shapeValues.objectVal(doc);
         if (indexedShape == null)
           return false;
@@ -80,7 +80,7 @@ public class ShapePredicateValueSource extends ValueSource {
       }
 
       @Override
-      public Explanation explain(int doc) {
+      public Explanation explain(int doc) throws IOException {
         Explanation exp = super.explain(doc);
         List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
         details.add(shapeValues.explain(doc));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
index 7cab3fe..e1f8f7c 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
@@ -16,15 +16,14 @@
  */
 package org.apache.lucene.spatial.vector;
 
+import org.apache.lucene.index.NumericDocValues;
 import org.locationtech.spatial4j.distance.DistanceCalculator;
 import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
 import java.util.Map;
@@ -67,33 +66,47 @@ public class DistanceValueSource extends ValueSource {
 
     final NumericDocValues ptX = DocValues.getNumeric(reader, strategy.getFieldNameX());
     final NumericDocValues ptY = DocValues.getNumeric(reader, strategy.getFieldNameY());
-    final Bits validX =  DocValues.getDocsWithField(reader, strategy.getFieldNameX());
-    final Bits validY =  DocValues.getDocsWithField(reader, strategy.getFieldNameY());
 
     return new FunctionValues() {
 
+      private int lastDocID = -1;
+
       private final Point from = DistanceValueSource.this.from;
       private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
       private final double nullValue =
           (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
+      private double getDocValue(NumericDocValues values, int doc) throws IOException {
+        int curDocID = values.docID();
+        if (doc > curDocID) {
+          curDocID = values.advance(doc);
+        }
+        if (doc == curDocID) {
+          return Double.longBitsToDouble(values.longValue());
+        } else {
+          return 0.0;
+        }
+      }
+
       @Override
-      public float floatVal(int doc) {
+      public float floatVal(int doc) throws IOException {
         return (float) doubleVal(doc);
       }
 
       @Override
-      public double doubleVal(int doc) {
+      public double doubleVal(int doc) throws IOException {
         // make sure it has minX and area
-        if (validX.get(doc)) {
-          assert validY.get(doc);
-          return calculator.distance(from, Double.longBitsToDouble(ptX.get(doc)), Double.longBitsToDouble(ptY.get(doc))) * multiplier;
+        double x = getDocValue(ptX, doc);
+        if (ptX.docID() == doc) {
+          double y = getDocValue(ptY, doc);
+          assert ptY.docID() == doc;
+          return calculator.distance(from, x, y) * multiplier;
         }
         return nullValue;
       }
 
       @Override
-      public String toString(int doc) {
+      public String toString(int doc) throws IOException {
         return description() + "=" + floatVal(doc);
       }
     };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
index 99a1f94..a5344b7 100644
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
@@ -132,12 +132,16 @@ final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointMultiTermQu
             if (preApproved.get(docId)) {
               return true;
             } else {
-              sdv.setDocument(docId);
-              int count = sdv.count();
-              for (int i = 0; i < count; i++) {
-                long hash = sdv.valueAt(i);
-                if (termsEnum.postFilter(GeoPointField.decodeLatitude(hash), GeoPointField.decodeLongitude(hash))) {
-                  return true;
+              if (docId > sdv.docID()) {
+                sdv.advance(docId);
+              }
+              if (docId == sdv.docID()) {
+                int count = sdv.docValueCount();
+                for (int i = 0; i < count; i++) {
+                  long hash = sdv.nextValue();
+                  if (termsEnum.postFilter(GeoPointField.decodeLatitude(hash), GeoPointField.decodeLongitude(hash))) {
+                    return true;
+                  }
                 }
               }
               return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointDistanceComparator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointDistanceComparator.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointDistanceComparator.java
index 869e3a3..2a522cc 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointDistanceComparator.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointDistanceComparator.java
@@ -26,11 +26,10 @@ import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Scorer;
-
-import org.apache.lucene.spatial3d.geom.GeoDistanceShape;
-import org.apache.lucene.spatial3d.geom.XYZBounds;
 import org.apache.lucene.spatial3d.geom.DistanceStyle;
+import org.apache.lucene.spatial3d.geom.GeoDistanceShape;
 import org.apache.lucene.spatial3d.geom.PlanetModel;
+import org.apache.lucene.spatial3d.geom.XYZBounds;
 
 /**
  * Compares documents by distance from an origin point, using a GeoDistanceShape to compute the distance
@@ -90,16 +89,19 @@ class Geo3DPointDistanceComparator extends FieldComparator<Double> implements Le
   
   @Override
   public int compareBottom(int doc) throws IOException {
-    currentDocs.setDocument(doc);
-
-    int numValues = currentDocs.count();
-    if (numValues == 0) {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
+    if (doc < currentDocs.docID()) {
       return Double.compare(bottomDistance, Double.POSITIVE_INFINITY);
     }
+    
+    int numValues = currentDocs.docValueCount();
+    assert numValues > 0;
 
     int cmp = -1;
     for (int i = 0; i < numValues; i++) {
-      long encoded = currentDocs.valueAt(i);
+      long encoded = currentDocs.nextValue();
 
       // Test against bounds.
       // First we need to decode...
@@ -148,19 +150,22 @@ class Geo3DPointDistanceComparator extends FieldComparator<Double> implements Le
     return Double.compare(topValue, computeMinimumDistance(doc));
   }
 
-  double computeMinimumDistance(final int doc) {
-    currentDocs.setDocument(doc);
+  double computeMinimumDistance(final int doc) throws IOException {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
     double minValue = Double.POSITIVE_INFINITY;
-    final int numValues = currentDocs.count();
-    for (int i = 0; i < numValues; i++) {
-      final long encoded = currentDocs.valueAt(i);
-      final double distance = distanceShape.computeDistance(DistanceStyle.ARC,
-        Geo3DDocValuesField.decodeXValue(encoded),
-        Geo3DDocValuesField.decodeYValue(encoded),
-        Geo3DDocValuesField.decodeZValue(encoded));
-      minValue = Math.min(minValue, distance);
+    if (doc == currentDocs.docID()) {
+      final int numValues = currentDocs.docValueCount();
+      for (int i = 0; i < numValues; i++) {
+        final long encoded = currentDocs.nextValue();
+        final double distance = distanceShape.computeDistance(DistanceStyle.ARC,
+                                                              Geo3DDocValuesField.decodeXValue(encoded),
+                                                              Geo3DDocValuesField.decodeYValue(encoded),
+                                                              Geo3DDocValuesField.decodeZValue(encoded));
+        minValue = Math.min(minValue, distance);
+      }
     }
     return minValue;
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointOutsideDistanceComparator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointOutsideDistanceComparator.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointOutsideDistanceComparator.java
index 10e0010..3ff8078 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointOutsideDistanceComparator.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/Geo3DPointOutsideDistanceComparator.java
@@ -26,9 +26,8 @@ import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Scorer;
-
-import org.apache.lucene.spatial3d.geom.GeoOutsideDistance;
 import org.apache.lucene.spatial3d.geom.DistanceStyle;
+import org.apache.lucene.spatial3d.geom.GeoOutsideDistance;
 import org.apache.lucene.spatial3d.geom.PlanetModel;
 
 /**
@@ -70,16 +69,19 @@ class Geo3DPointOutsideDistanceComparator extends FieldComparator<Double> implem
   
   @Override
   public int compareBottom(int doc) throws IOException {
-    currentDocs.setDocument(doc);
-
-    int numValues = currentDocs.count();
-    if (numValues == 0) {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
+    if (doc < currentDocs.docID()) {
       return Double.compare(bottomDistance, Double.POSITIVE_INFINITY);
     }
 
+    int numValues = currentDocs.docValueCount();
+    assert numValues > 0;
+
     int cmp = -1;
     for (int i = 0; i < numValues; i++) {
-      long encoded = currentDocs.valueAt(i);
+      long encoded = currentDocs.nextValue();
 
       // Test against bounds.
       // First we need to decode...
@@ -119,19 +121,22 @@ class Geo3DPointOutsideDistanceComparator extends FieldComparator<Double> implem
     return Double.compare(topValue, computeMinimumDistance(doc));
   }
 
-  double computeMinimumDistance(final int doc) {
-    currentDocs.setDocument(doc);
+  double computeMinimumDistance(final int doc) throws IOException {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
     double minValue = Double.POSITIVE_INFINITY;
-    final int numValues = currentDocs.count();
-    for (int i = 0; i < numValues; i++) {
-      final long encoded = currentDocs.valueAt(i);
-      final double distance = distanceShape.computeOutsideDistance(DistanceStyle.ARC,
-        Geo3DDocValuesField.decodeXValue(encoded),
-        Geo3DDocValuesField.decodeYValue(encoded),
-        Geo3DDocValuesField.decodeZValue(encoded));
-      minValue = Math.min(minValue, distance);
+    if (doc == currentDocs.docID()) {
+      final int numValues = currentDocs.docValueCount();
+      for (int i = 0; i < numValues; i++) {
+        final long encoded = currentDocs.nextValue();
+        final double distance = distanceShape.computeOutsideDistance(DistanceStyle.ARC,
+                                                                     Geo3DDocValuesField.decodeXValue(encoded),
+                                                                     Geo3DDocValuesField.decodeYValue(encoded),
+                                                                     Geo3DDocValuesField.decodeZValue(encoded));
+        minValue = Math.min(minValue, distance);
+      }
     }
     return minValue;
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
index d9baf61..450c526 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -791,8 +791,6 @@ public class TestGeo3DPoint extends LuceneTestCase {
 
     final int iters = atLeast(100);
 
-    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-
     for (int iter=0;iter<iters;iter++) {
 
       /*
@@ -835,8 +833,11 @@ public class TestGeo3DPoint extends LuceneTestCase {
         System.err.println("  hitCount: " + hits.cardinality());
       }
       
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+
       for(int docID=0;docID<r.maxDoc();docID++) {
-        int id = (int) docIDToID.get(docID);
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
         GeoPoint point = points[id];
         GeoPoint unquantizedPoint = unquantizedPoints[id];
         if (point != null && unquantizedPoint != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
index 9006ae2..a8dd507 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
@@ -239,12 +239,20 @@ public class DocumentDictionary implements Dictionary {
      * or if it's indexed as {@link NumericDocValues} (using <code>docId</code>) for the document.
      * If no value is found, then the weight is 0.
      */
-    protected long getWeight(Document doc, int docId) {
+    protected long getWeight(Document doc, int docId) throws IOException {
       IndexableField weight = doc.getField(weightField);
       if (weight != null) { // found weight as stored
         return (weight.numericValue() != null) ? weight.numericValue().longValue() : 0;
       } else if (weightValues != null) {  // found weight as NumericDocValue
-        return weightValues.get(docId);
+        if (weightValues.docID() < docId) {
+          weightValues.advance(docId);
+        }
+        if (weightValues.docID() == docId) {
+          return weightValues.longValue();
+        } else {
+          // missing
+          return 0;
+        }
       } else { // fall back
         return 0;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index 8affcc5..2c0b8f4 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -132,7 +132,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
      * by the <code>weightsValueSource</code>
      * */
     @Override
-    protected long getWeight(Document doc, int docId) {    
+    protected long getWeight(Document doc, int docId) {
       if (currentWeightValues == null) {
         return 0;
       }
@@ -145,7 +145,11 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
           throw new RuntimeException(e);
         }
       }
-      return currentWeightValues.longVal(docId - starts[subIndex]);
+      try {
+        return currentWeightValues.longVal(docId - starts[subIndex]);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 9174137..d05c39f 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -620,22 +620,27 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
                                              boolean doHighlight, Set<String> matchedTokens, String prefixToken)
       throws IOException {
 
-    BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
-
-    // This will just be null if app didn't pass payloads to build():
-    // TODO: maybe just stored fields?  they compress...
-    BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
     List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
     List<LookupResult> results = new ArrayList<>();
     for (int i=0;i<hits.scoreDocs.length;i++) {
       FieldDoc fd = (FieldDoc) hits.scoreDocs[i];
-      BytesRef term = textDV.get(fd.doc);
+      BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
+      textDV.advance(fd.doc);
+      BytesRef term = textDV.binaryValue();
       String text = term.utf8ToString();
       long score = (Long) fd.fields[0];
 
+      // This will just be null if app didn't pass payloads to build():
+      // TODO: maybe just stored fields?  they compress...
+      BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
+
       BytesRef payload;
       if (payloadsDV != null) {
-        payload = BytesRef.deepCopyOf(payloadsDV.get(fd.doc));
+        if (payloadsDV.advance(fd.doc) == fd.doc) {
+          payload = BytesRef.deepCopyOf(payloadsDV.binaryValue());
+        } else {
+          payload = new BytesRef(BytesRef.EMPTY_BYTES);
+        }
       } else {
         payload = null;
       }
@@ -646,11 +651,13 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       Set<BytesRef> contexts;
       if (contextsDV != null) {
         contexts = new HashSet<BytesRef>();
-        contextsDV.setDocument(fd.doc - leaves.get(segment).docBase);
-        long ord;
-        while ((ord = contextsDV.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-          BytesRef context = BytesRef.deepCopyOf(contextsDV.lookupOrd(ord));
-          contexts.add(context);
+        int targetDocID = fd.doc - leaves.get(segment).docBase;
+        if (contextsDV.advance(targetDocID) == targetDocID) {
+          long ord;
+          while ((ord = contextsDV.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+            BytesRef context = BytesRef.deepCopyOf(contextsDV.lookupOrd(ord));
+            contexts.add(context);
+          }
         }
       } else {
         contexts = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
index 925f8b0..413d401 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
@@ -28,9 +28,9 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanClause;
@@ -186,13 +186,6 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
                                                     boolean doHighlight, Set<String> matchedTokens, String prefixToken)
       throws IOException {
 
-    BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
-    assert textDV != null;
-
-    // This will just be null if app didn't pass payloads to build():
-    // TODO: maybe just stored fields?  they compress...
-    BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
-
     TreeSet<Lookup.LookupResult> results = new TreeSet<>(LOOKUP_COMP);
 
     // we reduce the num to the one initially requested
@@ -201,12 +194,25 @@ public class BlendedInfixSuggester extends AnalyzingInfixSuggester {
     for (int i = 0; i < hits.scoreDocs.length; i++) {
       FieldDoc fd = (FieldDoc) hits.scoreDocs[i];
 
-      final String text = textDV.get(fd.doc).utf8ToString();
+      BinaryDocValues textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME);
+      assert textDV != null;
+
+      textDV.advance(fd.doc);
+
+      final String text = textDV.binaryValue().utf8ToString();
       long weight = (Long) fd.fields[0];
 
+      // This will just be null if app didn't pass payloads to build():
+      // TODO: maybe just stored fields?  they compress...
+      BinaryDocValues payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads");
+
       BytesRef payload;
       if (payloadsDV != null) {
-        payload = BytesRef.deepCopyOf(payloadsDV.get(fd.doc));
+        if (payloadsDV.advance(fd.doc) == fd.doc) {
+          payload = BytesRef.deepCopyOf(payloadsDV.binaryValue());
+        } else {
+          payload = new BytesRef(BytesRef.EMPTY_BYTES);
+        }
       } else {
         payload = null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
index 777c0c8..20561b3 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
@@ -34,11 +34,13 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.suggest.BitsProducer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 import static org.apache.lucene.search.suggest.document.TestSuggestField.Entry;
 import static org.apache.lucene.search.suggest.document.TestSuggestField.assertSuggestions;
 import static org.apache.lucene.search.suggest.document.TestSuggestField.iwcWithSuggestField;
@@ -81,30 +83,21 @@ public class TestPrefixCompletionQuery extends LuceneTestCase {
     @Override
     public Bits getBits(final LeafReaderContext context) throws IOException {
       final int maxDoc = context.reader().maxDoc();
+      FixedBitSet bits = new FixedBitSet(maxDoc);
       final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field);
-      return new Bits() {
-
-        @Override
-        public boolean get(int doc) {
-          values.setDocument(doc);
-          final int count = values.count();
-          for (int i = 0; i < count; ++i) {
-            final long v = values.valueAt(i);
-            if (v >= min && v <= max) {
-              return true;
-            }
+      int docID;
+      while ((docID = values.nextDoc()) != NO_MORE_DOCS) {
+        final int count = values.docValueCount();
+        for (int i = 0; i < count; ++i) {
+          final long v = values.nextValue();
+          if (v >= min && v <= max) {
+            bits.set(docID);
+            break;
           }
-          return false;
         }
-
-        @Override
-        public int length() {
-          return maxDoc;
-        }
-        
-      };
+      }
+      return bits;
     }
-
   }
 
   public Directory dir;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
index e0af9a1..76ab1df 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
@@ -18,31 +18,28 @@ package org.apache.lucene.codecs.asserting;
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Iterator;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.AssertingLeafReader;
-import org.apache.lucene.index.AssertingLeafReader.AssertingRandomAccessOrds;
-import org.apache.lucene.index.AssertingLeafReader.AssertingSortedSetDocValues;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomAccessOrds;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.TestUtil;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 /**
  * Just like the default but with additional asserts.
  */
@@ -78,131 +75,138 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
     }
 
     @Override
-    public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
-      int count = 0;
-      for (Number v : values) {
-        count++;
+    public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      NumericDocValues values = valuesProducer.getNumeric(field);
+
+      int docID;
+      int lastDocID = -1;
+      while ((docID = values.nextDoc()) != NO_MORE_DOCS) {
+        assert docID >= 0 && docID < maxDoc;
+        assert docID > lastDocID;
+        lastDocID = docID;
+        long value = values.longValue();
       }
-      assert count == maxDoc;
-      TestUtil.checkIterator(values.iterator(), maxDoc, true);
-      in.addNumericField(field, values);
+      
+      in.addNumericField(field, valuesProducer);
     }
     
     @Override
-    public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
-      int count = 0;
-      for (BytesRef b : values) {
-        assert b == null || b.isValid();
-        count++;
+    public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      BinaryDocValues values = valuesProducer.getBinary(field);
+      
+      int docID;
+      int lastDocID = -1;
+      while ((docID = values.nextDoc()) != NO_MORE_DOCS) {
+        assert docID >= 0 && docID < maxDoc;
+        assert docID > lastDocID;
+        lastDocID = docID;
+        BytesRef value = values.binaryValue();
+        assert value.isValid();
       }
-      assert count == maxDoc;
-      TestUtil.checkIterator(values.iterator(), maxDoc, true);
-      in.addBinaryField(field, values);
+
+      in.addBinaryField(field, valuesProducer);
     }
     
     @Override
-    public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
-      int valueCount = 0;
+    public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      SortedDocValues values = valuesProducer.getSorted(field);
+
+      int valueCount = values.getValueCount();
+      assert valueCount <= maxDoc;
       BytesRef lastValue = null;
-      for (BytesRef b : values) {
+      for (int ord=0;ord<valueCount;ord++) {
+        BytesRef b = values.lookupOrd(ord);
         assert b != null;
         assert b.isValid();
-        if (valueCount > 0) {
+        if (ord > 0) {
           assert b.compareTo(lastValue) > 0;
         }
         lastValue = BytesRef.deepCopyOf(b);
-        valueCount++;
       }
-      assert valueCount <= maxDoc;
       
       FixedBitSet seenOrds = new FixedBitSet(valueCount);
       
-      int count = 0;
-      for (Number v : docToOrd) {
-        assert v != null;
-        int ord = v.intValue();
-        assert ord >= -1 && ord < valueCount;
-        if (ord >= 0) {
-          seenOrds.set(ord);
-        }
-        count++;
+      int docID;
+      int lastDocID = -1;
+      while ((docID = values.nextDoc()) != NO_MORE_DOCS) {
+        assert docID >= 0 && docID < maxDoc;
+        assert docID > lastDocID;
+        lastDocID = docID;
+        int ord = values.ordValue();
+        assert ord >= 0 && ord < valueCount;
+        seenOrds.set(ord);
       }
       
-      assert count == maxDoc;
       assert seenOrds.cardinality() == valueCount;
-      TestUtil.checkIterator(values.iterator(), valueCount, false);
-      TestUtil.checkIterator(docToOrd.iterator(), maxDoc, false);
-      in.addSortedField(field, values, docToOrd);
+      in.addSortedField(field, valuesProducer);
     }
     
     @Override
-    public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+    public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+
       long valueCount = 0;
-      Iterator<Number> valueIterator = values.iterator();
-      for (Number count : docToValueCount) {
-        assert count != null;
-        assert count.intValue() >= 0;
-        valueCount += count.intValue();
+      int lastDocID = -1;
+      while (true) {
+        int docID = values.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
+        assert values.docID() > lastDocID;
+        lastDocID = values.docID();
+        int count = values.docValueCount();
+        assert count > 0;
+        valueCount += count;
         long previous = Long.MIN_VALUE;
-        for (int i = 0; i < count.intValue(); i++) {
-          assert valueIterator.hasNext();
-          Number next = valueIterator.next();
-          assert next != null;
-          long nextValue = next.longValue();
+        for (int i = 0; i < count; i++) {
+          long nextValue = values.nextValue();
           assert nextValue >= previous;
           previous = nextValue;
         }
       }
-      assert valueIterator.hasNext() == false;
-      TestUtil.checkIterator(docToValueCount.iterator(), maxDoc, false);
-      TestUtil.checkIterator(values.iterator(), valueCount, false);
-      in.addSortedNumericField(field, docToValueCount, values);
+      in.addSortedNumericField(field, valuesProducer);
     }
     
     @Override
-    public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
-      long valueCount = 0;
+    public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      SortedSetDocValues values = valuesProducer.getSortedSet(field);
+
+      long valueCount = values.getValueCount();
       BytesRef lastValue = null;
-      for (BytesRef b : values) {
+      for (long i=0;i<valueCount;i++) {
+        BytesRef b = values.lookupOrd(i);
         assert b != null;
         assert b.isValid();
-        if (valueCount > 0) {
+        if (i > 0) {
           assert b.compareTo(lastValue) > 0;
         }
         lastValue = BytesRef.deepCopyOf(b);
-        valueCount++;
       }
       
       int docCount = 0;
-      long ordCount = 0;
       LongBitSet seenOrds = new LongBitSet(valueCount);
-      Iterator<Number> ordIterator = ords.iterator();
-      for (Number v : docToOrdCount) {
-        assert v != null;
-        int count = v.intValue();
-        assert count >= 0;
+      while (true) {
+        int docID = values.nextDoc();
+        if (docID == NO_MORE_DOCS) {
+          break;
+        }
         docCount++;
-        ordCount += count;
         
         long lastOrd = -1;
-        for (int i = 0; i < count; i++) {
-          Number o = ordIterator.next();
-          assert o != null;
-          long ord = o.longValue();
-          assert ord >= 0 && ord < valueCount;
+        while (true) {
+          long ord = values.nextOrd();
+          if (ord == SortedSetDocValues.NO_MORE_ORDS) {
+            break;
+          }
+          assert ord >= 0 && ord < valueCount: "ord=" + ord + " is not in bounds 0 .." + (valueCount-1);
           assert ord > lastOrd : "ord=" + ord + ",lastOrd=" + lastOrd;
           seenOrds.set(ord);
           lastOrd = ord;
         }
       }
-      assert ordIterator.hasNext() == false;
       
-      assert docCount == maxDoc;
       assert seenOrds.cardinality() == valueCount;
-      TestUtil.checkIterator(values.iterator(), valueCount, false);
-      TestUtil.checkIterator(docToOrdCount.iterator(), maxDoc, false);
-      TestUtil.checkIterator(ords.iterator(), ordCount, false);
-      in.addSortedSetField(field, values, docToOrdCount, ords);
+      in.addSortedSetField(field, valuesProducer);
     }
     
     @Override
@@ -262,23 +266,10 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
       assert field.getDocValuesType() == DocValuesType.SORTED_SET;
       SortedSetDocValues values = in.getSortedSet(field);
       assert values != null;
-      if (values instanceof RandomAccessOrds) {
-        return new AssertingRandomAccessOrds((RandomAccessOrds) values, maxDoc);
-      } else {
-        return new AssertingSortedSetDocValues(values, maxDoc);
-      }
+      return new AssertingLeafReader.AssertingSortedSetDocValues(values, maxDoc);
     }
     
     @Override
-    public Bits getDocsWithField(FieldInfo field) throws IOException {
-      assert field.getDocValuesType() != DocValuesType.NONE;
-      Bits bits = in.getDocsWithField(field);
-      assert bits != null;
-      assert bits.length() == maxDoc;
-      return new AssertingLeafReader.AssertingBits(bits);
-    }
-
-    @Override
     public void close() throws IOException {
       in.close();
       in.close(); // close again

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
index 4837513..f4abb54 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
@@ -106,7 +106,7 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
     @Override
     public boolean get(int index) {
       assert index >= 0;
-      assert index < in.length();
+      assert index < in.length(): "index=" + index + " vs in.length()=" + in.length();
       return in.get(index);
     }
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
index e646eb5..bf830bf 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
@@ -30,6 +30,8 @@ import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.TestUtil;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 /**
  * Just like the default but with additional asserts.
  */
@@ -61,15 +63,19 @@ public class AssertingNormsFormat extends NormsFormat {
     }
 
     @Override
-    public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
-      int count = 0;
-      for (Number v : values) {
-        assert v != null;
-        count++;
+    public void addNormsField(FieldInfo field, NormsProducer valuesProducer) throws IOException {
+      NumericDocValues values = valuesProducer.getNorms(field);
+
+      int docID;
+      int lastDocID = -1;
+      while ((docID = values.nextDoc()) != NO_MORE_DOCS) {
+        assert docID >= 0 && docID < maxDoc;
+        assert docID > lastDocID;
+        lastDocID = docID;
+        long value = values.longValue();
       }
-      assert count == maxDoc;
-      TestUtil.checkIterator(values.iterator(), maxDoc, false);
-      in.addNormsField(field, values);
+
+      in.addNormsField(field, valuesProducer);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java
index f29a3f9..10ae536 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java
@@ -25,7 +25,6 @@ import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.BytesRef;
 
 class CrankyDocValuesFormat extends DocValuesFormat {
   final DocValuesFormat delegate;
@@ -70,43 +69,43 @@ class CrankyDocValuesFormat extends DocValuesFormat {
     }
 
     @Override
-    public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+    public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from DocValuesConsumer.addNumericField()");
       }
-      delegate.addNumericField(field, values);
+      delegate.addNumericField(field, valuesProducer);
     }
 
     @Override
-    public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+    public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from DocValuesConsumer.addBinaryField()");
       }
-      delegate.addBinaryField(field, values);
+      delegate.addBinaryField(field, valuesProducer);
     }
 
     @Override
-    public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from DocValuesConsumer.addSortedField()");
       }
-      delegate.addSortedField(field, values, docToOrd);
+      delegate.addSortedField(field, valuesProducer);
     }
     
     @Override
-    public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+    public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from DocValuesConsumer.addSortedNumericField()");
       }
-      delegate.addSortedNumericField(field, docToValueCount, values);
+      delegate.addSortedNumericField(field, valuesProducer);
     }
 
     @Override
-    public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+    public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from DocValuesConsumer.addSortedSetField()");
       }
-      delegate.addSortedSetField(field, values, docToOrdCount, ords);
+      delegate.addSortedSetField(field, valuesProducer);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyNormsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyNormsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyNormsFormat.java
index 34b606a..9168ce0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyNormsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyNormsFormat.java
@@ -66,11 +66,11 @@ class CrankyNormsFormat extends NormsFormat {
     }
 
     @Override
-    public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
+    public void addNormsField(FieldInfo field, NormsProducer valuesProducer) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from NormsConsumer.addNormsField()");
       }
-      delegate.addNormsField(field, values);
+      delegate.addNormsField(field, valuesProducer);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
index 275c186..19b4a1c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseGeoPointTestCase.java
@@ -37,9 +37,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.geo.Rectangle;
-import org.apache.lucene.geo.GeoUtils;
-import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -792,8 +789,6 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
 
     int iters = atLeast(25);
 
-    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-
     Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
     int maxDoc = s.getIndexReader().maxDoc();
 
@@ -833,8 +828,10 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         });
 
       boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
       for(int docID=0;docID<maxDoc;docID++) {
-        int id = (int) docIDToID.get(docID);
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
         boolean expected;
         if (liveDocs != null && liveDocs.get(docID) == false) {
           // document is deleted
@@ -920,8 +917,6 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
 
     int iters = atLeast(25);
 
-    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-
     Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
     int maxDoc = s.getIndexReader().maxDoc();
 
@@ -971,8 +966,10 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         });
 
       boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
       for(int docID=0;docID<maxDoc;docID++) {
-        int id = (int) docIDToID.get(docID);
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
         boolean expected;
         if (liveDocs != null && liveDocs.get(docID) == false) {
           // document is deleted
@@ -1062,8 +1059,6 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
 
     final int iters = atLeast(75);
 
-    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-
     Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
     int maxDoc = s.getIndexReader().maxDoc();
 
@@ -1103,8 +1098,10 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
         });
 
       boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
       for(int docID=0;docID<maxDoc;docID++) {
-        int id = (int) docIDToID.get(docID);
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
         boolean expected;
         if (liveDocs != null && liveDocs.get(docID) == false) {
           // document is deleted

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
index c33db57..a109eb8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
@@ -396,18 +396,64 @@ public class AssertingLeafReader extends FilterLeafReader {
     private final Thread creationThread = Thread.currentThread();
     private final NumericDocValues in;
     private final int maxDoc;
+    private int lastDocID = -1;
     
     public AssertingNumericDocValues(NumericDocValues in, int maxDoc) {
       this.in = in;
       this.maxDoc = maxDoc;
+      // should start unpositioned:
+      assert in.docID() == -1;
     }
 
     @Override
-    public long get(int docID) {
+    public int docID() {
+      assertThread("Numeric doc values", creationThread);
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      assertThread("Numeric doc values", creationThread);
+      int docID = in.nextDoc();
+      assert docID > lastDocID;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      assert docID == in.docID();
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
       assertThread("Numeric doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc;
-      return in.get(docID);
+      assert target >= 0;
+      assert target >= in.docID();
+      int docID = in.advance(target);
+      assert docID >= target;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      assertThread("Numeric doc values", creationThread);
+      long cost = in.cost();
+      assert cost >= 0;
+      return cost;
+    }
+
+    @Override
+    public long longValue() {
+      assertThread("Numeric doc values", creationThread);
+      assert in.docID() != -1;
+      assert in.docID() != NO_MORE_DOCS;
+      return in.longValue();
     }    
+
+    @Override
+    public String toString() {
+      return "AssertingNumericDocValues(" + in + ")";
+    }
   }
   
   /** Wraps a BinaryDocValues but with additional asserts */
@@ -415,28 +461,73 @@ public class AssertingLeafReader extends FilterLeafReader {
     private final Thread creationThread = Thread.currentThread();
     private final BinaryDocValues in;
     private final int maxDoc;
+    private int lastDocID = -1;
     
     public AssertingBinaryDocValues(BinaryDocValues in, int maxDoc) {
       this.in = in;
       this.maxDoc = maxDoc;
+      // should start unpositioned:
+      assert in.docID() == -1;
     }
 
     @Override
-    public BytesRef get(int docID) {
+    public int docID() {
       assertThread("Binary doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc;
-      final BytesRef result = in.get(docID);
-      assert result.isValid();
-      return result;
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      assertThread("Binary doc values", creationThread);
+      int docID = in.nextDoc();
+      assert docID > lastDocID;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      assert docID == in.docID();
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      assertThread("Binary doc values", creationThread);
+      assert target >= 0;
+      assert target >= in.docID();
+      int docID = in.advance(target);
+      assert docID >= target;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      assertThread("Binary doc values", creationThread);
+      long cost = in.cost();
+      assert cost >= 0;
+      return cost;
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      assertThread("Binary doc values", creationThread);
+      assert in.docID() != -1;
+      assert in.docID() != NO_MORE_DOCS;
+      return in.binaryValue();
+    }
+
+    @Override
+    public String toString() {
+      return "AssertingBinaryDocValues(" + in + ")";
     }
   }
-  
+
   /** Wraps a SortedDocValues but with additional asserts */
   public static class AssertingSortedDocValues extends SortedDocValues {
     private final Thread creationThread = Thread.currentThread();
     private final SortedDocValues in;
     private final int maxDoc;
     private final int valueCount;
+    private int lastDocID = -1;
     
     public AssertingSortedDocValues(SortedDocValues in, int maxDoc) {
       this.in = in;
@@ -446,10 +537,46 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public int getOrd(int docID) {
+    public int docID() {
       assertThread("Sorted doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc;
-      int ord = in.getOrd(docID);
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      assertThread("Sorted doc values", creationThread);
+      int docID = in.nextDoc();
+      assert docID > lastDocID;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      assert docID == in.docID();
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      assertThread("Sorted doc values", creationThread);
+      assert target >= 0;
+      assert target >= in.docID();
+      int docID = in.advance(target);
+      assert docID >= target;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      lastDocID = docID;
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      assertThread("Sorted doc values", creationThread);
+      long cost = in.cost();
+      assert cost >= 0;
+      return cost;
+    }
+
+    @Override
+    public int ordValue() {
+      assertThread("Sorted doc values", creationThread);
+      int ord = in.ordValue();
       assert ord >= -1 && ord < valueCount;
       return ord;
     }
@@ -472,10 +599,9 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public BytesRef get(int docID) {
+    public BytesRef binaryValue() {
       assertThread("Sorted doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc;
-      final BytesRef result = in.get(docID);
+      final BytesRef result = in.binaryValue();
       assert result.isValid();
       return result;
     }
@@ -491,11 +617,13 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
   }
   
-  /** Wraps a SortedSetDocValues but with additional asserts */
+  /** Wraps a SortedNumericDocValues but with additional asserts */
   public static class AssertingSortedNumericDocValues extends SortedNumericDocValues {
     private final Thread creationThread = Thread.currentThread();
     private final SortedNumericDocValues in;
     private final int maxDoc;
+    private int lastDocID = -1;
+    private int valueUpto;
     
     public AssertingSortedNumericDocValues(SortedNumericDocValues in, int maxDoc) {
       this.in = in;
@@ -503,110 +631,58 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public void setDocument(int doc) {
-      assertThread("Sorted numeric doc values", creationThread);
-      assert doc >= 0 && doc < maxDoc;
-      in.setDocument(doc);
-      // check the values are actually sorted
-      long previous = Long.MIN_VALUE;
-      for (int i = 0; i < in.count(); i++) {
-        long v = in.valueAt(i);
-        assert v >= previous;
-        previous = v;
-      }
-    }
-
-    @Override
-    public long valueAt(int index) {
-      assertThread("Sorted numeric doc values", creationThread);
-      assert index < in.count();
-      return in.valueAt(index);
-    }
-
-    @Override
-    public int count() {
-      assertThread("Sorted numeric doc values", creationThread);
-      return in.count();
-    } 
-  }
-  
-  /** Wraps a RandomAccessOrds but with additional asserts */
-  public static class AssertingRandomAccessOrds extends RandomAccessOrds {
-    private final Thread creationThread = Thread.currentThread();
-    private final RandomAccessOrds in;
-    private final int maxDoc;
-    private final long valueCount;
-    long lastOrd = NO_MORE_ORDS;
-    
-    public AssertingRandomAccessOrds(RandomAccessOrds in, int maxDoc) {
-      this.in = in;
-      this.maxDoc = maxDoc;
-      this.valueCount = in.getValueCount();
-      assert valueCount >= 0;
-    }
-    
-    @Override
-    public long nextOrd() {
-      assertThread("Sorted numeric doc values", creationThread);
-      assert lastOrd != NO_MORE_ORDS;
-      long ord = in.nextOrd();
-      assert ord < valueCount;
-      assert ord == NO_MORE_ORDS || ord > lastOrd;
-      lastOrd = ord;
-      return ord;
-    }
-
-    @Override
-    public void setDocument(int docID) {
-      assertThread("Sorted numeric doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc : "docid=" + docID + ",maxDoc=" + maxDoc;
-      in.setDocument(docID);
-      lastOrd = -2;
+    public int docID() {
+      return in.docID();
     }
 
     @Override
-    public BytesRef lookupOrd(long ord) {
+    public int nextDoc() throws IOException {
       assertThread("Sorted numeric doc values", creationThread);
-      assert ord >= 0 && ord < valueCount;
-      final BytesRef result = in.lookupOrd(ord);
-      assert result.isValid();
-      return result;
+      int docID = in.nextDoc();
+      assert docID > lastDocID;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      assert docID == in.docID();
+      lastDocID = docID;
+      valueUpto = 0;
+      return docID;
     }
 
     @Override
-    public long getValueCount() {
+    public int advance(int target) throws IOException {
       assertThread("Sorted numeric doc values", creationThread);
-      long valueCount = in.getValueCount();
-      assert valueCount == this.valueCount; // should not change
-      return valueCount;
+      assert target >= 0;
+      assert target >= in.docID();
+      int docID = in.advance(target);
+      assert docID == in.docID();
+      assert docID >= target;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      lastDocID = docID;
+      valueUpto = 0;
+      return docID;
     }
 
     @Override
-    public long lookupTerm(BytesRef key) {
+    public long cost() {
       assertThread("Sorted numeric doc values", creationThread);
-      assert key.isValid();
-      long result = in.lookupTerm(key);
-      assert result < valueCount;
-      assert key.isValid();
-      return result;
+      long cost = in.cost();
+      assert cost >= 0;
+      return cost;
     }
 
     @Override
-    public long ordAt(int index) {
+    public long nextValue() throws IOException {
       assertThread("Sorted numeric doc values", creationThread);
-      assert index < in.cardinality();
-      long ord = in.ordAt(index);
-      assert ord >= 0 && ord < valueCount;
-      return ord;
+      assert valueUpto < in.docValueCount(): "valueUpto=" + valueUpto + " in.docValueCount()=" + in.docValueCount();
+      valueUpto++;
+      return in.nextValue();
     }
 
     @Override
-    public int cardinality() {
+    public int docValueCount() {
       assertThread("Sorted numeric doc values", creationThread);
-      int cardinality = in.cardinality();
-      assert cardinality >= 0;
-      return cardinality;
-    }
+      assert in.docValueCount() > 0;
+      return in.docValueCount();
+    } 
   }
   
   /** Wraps a SortedSetDocValues but with additional asserts */
@@ -615,6 +691,7 @@ public class AssertingLeafReader extends FilterLeafReader {
     private final SortedSetDocValues in;
     private final int maxDoc;
     private final long valueCount;
+    private int lastDocID = -1;
     long lastOrd = NO_MORE_ORDS;
     
     public AssertingSortedSetDocValues(SortedSetDocValues in, int maxDoc) {
@@ -623,10 +700,50 @@ public class AssertingLeafReader extends FilterLeafReader {
       this.valueCount = in.getValueCount();
       assert valueCount >= 0;
     }
+
+    @Override
+    public int docID() {
+      assertThread("Sorted set doc values", creationThread);
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      assertThread("Sorted set doc values", creationThread);
+      int docID = in.nextDoc();
+      assert docID > lastDocID;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      assert docID == in.docID();
+      lastDocID = docID;
+      lastOrd = -2;
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      assertThread("Sorted set doc values", creationThread);
+      assert target >= 0;
+      assert target >= in.docID();
+      int docID = in.advance(target);
+      assert docID == in.docID();
+      assert docID >= target;
+      assert docID == NO_MORE_DOCS || docID < maxDoc;
+      lastDocID = docID;
+      lastOrd = -2;
+      return docID;
+    }
+
+    @Override
+    public long cost() {
+      assertThread("Sorted set doc values", creationThread);
+      long cost = in.cost();
+      assert cost >= 0;
+      return cost;
+    }
     
     @Override
-    public long nextOrd() {
-      assertThread("Sorted numeric doc values", creationThread);
+    public long nextOrd() throws IOException {
+      assertThread("Sorted set doc values", creationThread);
       assert lastOrd != NO_MORE_ORDS;
       long ord = in.nextOrd();
       assert ord < valueCount;
@@ -636,16 +753,8 @@ public class AssertingLeafReader extends FilterLeafReader {
     }
 
     @Override
-    public void setDocument(int docID) {
-      assertThread("Sorted numeric doc values", creationThread);
-      assert docID >= 0 && docID < maxDoc : "docid=" + docID + ",maxDoc=" + maxDoc;
-      in.setDocument(docID);
-      lastOrd = -2;
-    }
-
-    @Override
     public BytesRef lookupOrd(long ord) {
-      assertThread("Sorted numeric doc values", creationThread);
+      assertThread("Sorted set doc values", creationThread);
       assert ord >= 0 && ord < valueCount;
       final BytesRef result = in.lookupOrd(ord);
       assert result.isValid();
@@ -654,7 +763,7 @@ public class AssertingLeafReader extends FilterLeafReader {
 
     @Override
     public long getValueCount() {
-      assertThread("Sorted numeric doc values", creationThread);
+      assertThread("Sorted set doc values", creationThread);
       long valueCount = in.getValueCount();
       assert valueCount == this.valueCount; // should not change
       return valueCount;
@@ -662,7 +771,7 @@ public class AssertingLeafReader extends FilterLeafReader {
 
     @Override
     public long lookupTerm(BytesRef key) {
-      assertThread("Sorted numeric doc values", creationThread);
+      assertThread("Sorted set doc values", creationThread);
       assert key.isValid();
       long result = in.lookupTerm(key);
       assert result < valueCount;
@@ -734,11 +843,7 @@ public class AssertingLeafReader extends FilterLeafReader {
     if (dv != null) {
       assert fi != null;
       assert fi.getDocValuesType() == DocValuesType.SORTED_SET;
-      if (dv instanceof RandomAccessOrds) {
-        return new AssertingRandomAccessOrds((RandomAccessOrds) dv, maxDoc());
-      } else {
-        return new AssertingSortedSetDocValues(dv, maxDoc());
-      }
+      return new AssertingSortedSetDocValues(dv, maxDoc());
     } else {
       assert fi == null || fi.getDocValuesType() != DocValuesType.SORTED_SET;
       return null;
@@ -795,21 +900,6 @@ public class AssertingLeafReader extends FilterLeafReader {
     return liveDocs;
   }
 
-  @Override
-  public Bits getDocsWithField(String field) throws IOException {
-    Bits docsWithField = super.getDocsWithField(field);
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (docsWithField != null) {
-      assert fi != null;
-      assert fi.getDocValuesType() != DocValuesType.NONE;
-      assert maxDoc() == docsWithField.length();
-      docsWithField = new AssertingBits(docsWithField);
-    } else {
-      assert fi == null || fi.getDocValuesType() == DocValuesType.NONE;
-    }
-    return docsWithField;
-  }
-
   // we don't change behavior of the reader: just validate the API.
 
   @Override