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

[08/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/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
index 83e01e4..93edcc0 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java
@@ -113,10 +113,10 @@ final class GlobalOrdinalsQuery extends Query {
         return Explanation.noMatch("Not a match");
       }
 
-      int segmentOrd = values.getOrd(doc);
-      if (segmentOrd == -1) {
+      if (values.advance(doc) != doc) {
         return Explanation.noMatch("Not a match");
       }
+      int segmentOrd = values.ordValue();
       BytesRef joinValue = values.lookupOrd(segmentOrd);
 
       int ord;
@@ -170,8 +170,12 @@ final class GlobalOrdinalsQuery extends Query {
 
         @Override
         public boolean matches() throws IOException {
-          final long segmentOrd = values.getOrd(approximation.docID());
-          if (segmentOrd != -1) {
+          int docID = approximation.docID();
+          if (docID > values.docID()) {
+            values.advance(docID);
+          }
+          if (docID == values.docID()) {
+            final long segmentOrd = values.ordValue();
             final long globalOrd = segmentOrdToGlobalOrdLookup.get(segmentOrd);
             if (foundOrds.get(globalOrd)) {
               return true;
@@ -204,9 +208,12 @@ final class GlobalOrdinalsQuery extends Query {
 
         @Override
         public boolean matches() throws IOException {
-          final long segmentOrd = values.getOrd(approximation.docID());
-          if (segmentOrd != -1) {
-            if (foundOrds.get(segmentOrd)) {
+          int docID = approximation.docID();
+          if (docID > values.docID()) {
+            values.advance(docID);
+          }
+          if (docID == values.docID()) {
+            if (foundOrds.get(values.ordValue())) {
               return true;
             }
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreCollector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreCollector.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreCollector.java
index 57a8442..cc58041 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreCollector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreCollector.java
@@ -16,6 +16,9 @@
  */
 package org.apache.lucene.search.join;
 
+import java.io.IOException;
+import java.util.Arrays;
+
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
@@ -26,9 +29,6 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.LongValues;
 
-import java.io.IOException;
-import java.util.Arrays;
-
 abstract class GlobalOrdinalsWithScoreCollector implements Collector {
 
   final String field;
@@ -113,9 +113,11 @@ abstract class GlobalOrdinalsWithScoreCollector implements Collector {
 
     @Override
     public void collect(int doc) throws IOException {
-      final long segmentOrd = docTermOrds.getOrd(doc);
-      if (segmentOrd != -1) {
-        final int globalOrd = (int) segmentOrdToGlobalOrdLookup.get(segmentOrd);
+      if (doc > docTermOrds.docID()) {
+        docTermOrds.advance(doc);
+      }
+      if (doc == docTermOrds.docID()) {
+        final int globalOrd = (int) segmentOrdToGlobalOrdLookup.get(docTermOrds.ordValue());
         collectedOrds.set(globalOrd);
         float existingScore = scores.getScore(globalOrd);
         float newScore = scorer.score();
@@ -143,8 +145,11 @@ abstract class GlobalOrdinalsWithScoreCollector implements Collector {
 
     @Override
     public void collect(int doc) throws IOException {
-      final int segmentOrd = docTermOrds.getOrd(doc);
-      if (segmentOrd != -1) {
+      if (doc > docTermOrds.docID()) {
+        docTermOrds.advance(doc);
+      }
+      if (doc == docTermOrds.docID()) {
+        int segmentOrd = docTermOrds.ordValue();
         collectedOrds.set(segmentOrd);
         float existingScore = scores.getScore(segmentOrd);
         float newScore = scorer.score();
@@ -253,9 +258,11 @@ abstract class GlobalOrdinalsWithScoreCollector implements Collector {
 
           @Override
           public void collect(int doc) throws IOException {
-            final long segmentOrd = docTermOrds.getOrd(doc);
-            if (segmentOrd != -1) {
-              final int globalOrd = (int) segmentOrdToGlobalOrdLookup.get(segmentOrd);
+            if (doc > docTermOrds.docID()) {
+              docTermOrds.advance(doc);
+            }
+            if (doc == docTermOrds.docID()) {
+              final int globalOrd = (int) segmentOrdToGlobalOrdLookup.get(docTermOrds.ordValue());
               collectedOrds.set(globalOrd);
               occurrences.increment(globalOrd);
             }
@@ -269,8 +276,11 @@ abstract class GlobalOrdinalsWithScoreCollector implements Collector {
 
           @Override
           public void collect(int doc) throws IOException {
-            final int segmentOrd = docTermOrds.getOrd(doc);
-            if (segmentOrd != -1) {
+            if (doc > docTermOrds.docID()) {
+              docTermOrds.advance(doc);
+            }
+            if (doc == docTermOrds.docID()) {
+              int segmentOrd = docTermOrds.ordValue();
               collectedOrds.set(segmentOrd);
               occurrences.increment(segmentOrd);
             }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
index a192996..0aedf35 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
@@ -16,15 +16,18 @@
  */
 package org.apache.lucene.search.join;
 
+import java.io.IOException;
+import java.util.Set;
+
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.FilterWeight;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.FilterWeight;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -33,9 +36,6 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongValues;
 
-import java.io.IOException;
-import java.util.Set;
-
 final class GlobalOrdinalsWithScoreQuery extends Query {
 
   private final GlobalOrdinalsWithScoreCollector collector;
@@ -118,11 +118,11 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
       if (values == null) {
         return Explanation.noMatch("Not a match");
       }
-
-      int segmentOrd = values.getOrd(doc);
-      if (segmentOrd == -1) {
+      if (values.advance(doc) != doc) {
         return Explanation.noMatch("Not a match");
       }
+
+      int segmentOrd = values.ordValue();
       BytesRef joinValue = values.lookupOrd(segmentOrd);
 
       int ord;
@@ -175,8 +175,12 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
 
         @Override
         public boolean matches() throws IOException {
-          final long segmentOrd = values.getOrd(approximation.docID());
-          if (segmentOrd != -1) {
+          int docID = approximation.docID();
+          if (docID > values.docID()) {
+            values.advance(docID);
+          }
+          if (docID == values.docID()) {
+            final long segmentOrd = values.ordValue();
             final int globalOrd = (int) segmentOrdToGlobalOrdLookup.get(segmentOrd);
             if (collector.match(globalOrd)) {
               score = collector.score(globalOrd);
@@ -209,8 +213,12 @@ final class GlobalOrdinalsWithScoreQuery extends Query {
 
         @Override
         public boolean matches() throws IOException {
-          final int segmentOrd = values.getOrd(approximation.docID());
-          if (segmentOrd != -1) {
+          int docID = approximation.docID();
+          if (docID > values.docID()) {
+            values.advance(docID);
+          }
+          if (docID == values.docID()) {
+            final int segmentOrd = values.ordValue();
             if (collector.match(segmentOrd)) {
               score = collector.score(segmentOrd);
               return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
index b7c2a5b..bfc1f9b 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
@@ -102,12 +102,10 @@ public final class JoinUtil {
       termsWithScoreCollector = GenericTermsCollector.createCollectorMV(mvFunction, scoreMode);
     } else {
       Function<BinaryDocValues> svFunction = DocValuesTermsCollector.binaryDocValues(fromField);
-      termsWithScoreCollector =  GenericTermsCollector.createCollectorSV(svFunction, scoreMode);
+      termsWithScoreCollector = GenericTermsCollector.createCollectorSV(svFunction, scoreMode);
     }
     
-    return createJoinQuery(multipleValuesPerDocument, toField, fromQuery, fromSearcher, scoreMode,
-        termsWithScoreCollector);
-    
+    return createJoinQuery(multipleValuesPerDocument, toField, fromQuery, fromSearcher, scoreMode, termsWithScoreCollector);
   }
   
   /**
@@ -202,12 +200,16 @@ public final class JoinUtil {
 
         @Override
         public void collect(int doc) throws IOException {
-          sortedNumericDocValues.setDocument(doc);
-          for (int i = 0; i < sortedNumericDocValues.count(); i++) {
-            long value = sortedNumericDocValues.valueAt(i);
-            joinValues.add(value);
-            if (needsScore) {
-              scoreAggregator.accept(value, scorer.score());
+          if (doc > sortedNumericDocValues.docID()) {
+            sortedNumericDocValues.advance(doc);
+          }
+          if (doc == sortedNumericDocValues.docID()) {
+            for (int i = 0; i < sortedNumericDocValues.docValueCount(); i++) {
+              long value = sortedNumericDocValues.nextValue();
+              joinValues.add(value);
+              if (needsScore) {
+                scoreAggregator.accept(value, scorer.score());
+              }
             }
           }
         }
@@ -232,10 +234,29 @@ public final class JoinUtil {
 
         NumericDocValues numericDocValues;
         Scorer scorer;
+        private int lastDocID = -1;
+
+        private boolean docsInOrder(int docID) {
+          if (docID < lastDocID) {
+            throw new AssertionError("docs out of order: lastDocID=" + lastDocID + " vs docID=" + docID);
+          }
+          lastDocID = docID;
+          return true;
+        }
 
         @Override
         public void collect(int doc) throws IOException {
-          long value = numericDocValues.get(doc);
+          assert docsInOrder(doc);
+          int dvDocID = numericDocValues.docID();
+          if (dvDocID < doc) {
+            dvDocID = numericDocValues.advance(doc);
+          }
+          long value;
+          if (dvDocID == doc) {
+            value = numericDocValues.longValue();
+          } else {
+            value = 0;
+          }
           joinValues.add(value);
           if (needsScore) {
             scoreAggregator.accept(value, scorer.score());
@@ -245,6 +266,7 @@ public final class JoinUtil {
         @Override
         protected void doSetNextReader(LeafReaderContext context) throws IOException {
           numericDocValues = DocValues.getNumeric(context.reader(), fromField);
+          lastDocID = -1;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
index f2b3d42..8475af7 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
@@ -40,7 +40,6 @@ abstract class TermsCollector<DV> extends DocValuesTermsCollector<DV> {
     return collectorTerms;
   }
 
-  
   /**
    * Chooses the right {@link TermsCollector} implementation.
    *
@@ -64,10 +63,14 @@ abstract class TermsCollector<DV> extends DocValuesTermsCollector<DV> {
     @Override
     public void collect(int doc) throws IOException {
       long ord;
-      docValues.setDocument(doc);
-      while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        final BytesRef term = docValues.lookupOrd(ord);
-        collectorTerms.add(term);
+      if (doc > docValues.docID()) {
+        docValues.advance(doc);
+      }
+      if (doc == docValues.docID()) {
+        while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+          final BytesRef term = docValues.lookupOrd(ord);
+          collectorTerms.add(term);
+        }
       }
     }
   }
@@ -81,7 +84,15 @@ abstract class TermsCollector<DV> extends DocValuesTermsCollector<DV> {
 
     @Override
     public void collect(int doc) throws IOException {
-      final BytesRef term = docValues.get(doc);
+      if (docValues.docID() < doc) {
+        docValues.advance(doc);
+      }
+      BytesRef term;
+      if (docValues.docID() == doc) {
+        term = docValues.binaryValue();
+      } else {
+        term = new BytesRef(BytesRef.EMPTY_BYTES);
+      }
       collectorTerms.add(term);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
index a11dcec..22fc54d 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
@@ -23,6 +23,7 @@ import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 
 abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV> 
@@ -95,7 +96,16 @@ abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV>
 
     @Override
     public void collect(int doc) throws IOException {
-      int ord = collectedTerms.add(docValues.get(doc));
+      if (docValues.docID() < doc) {
+        docValues.advance(doc);
+      }
+      BytesRef value;
+      if (docValues.docID() == doc) {
+        value = docValues.binaryValue();
+      } else {
+        value = new BytesRef(BytesRef.EMPTY_BYTES);
+      }
+      int ord = collectedTerms.add(value);
       if (ord < 0) {
         ord = -ord - 1;
       } else {
@@ -145,7 +155,16 @@ abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV>
 
       @Override
       public void collect(int doc) throws IOException {
-        int ord = collectedTerms.add(docValues.get(doc));
+        if (docValues.docID() < doc) {
+          docValues.advance(doc);
+        }
+        BytesRef value;
+        if (docValues.docID() == doc) {
+          value = docValues.binaryValue();
+        } else {
+          value = new BytesRef(BytesRef.EMPTY_BYTES);
+        }
+        int ord = collectedTerms.add(value);
         if (ord < 0) {
           ord = -ord - 1;
         } else {
@@ -188,25 +207,28 @@ abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV>
 
     @Override
     public void collect(int doc) throws IOException {
-      docValues.setDocument(doc);
-      long ord;
-      while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        int termID = collectedTerms.add(docValues.lookupOrd(ord));
-        if (termID < 0) {
-          termID = -termID - 1;
-        } else {
-          if (termID >= scoreSums.length) {
-            int begin = scoreSums.length;
-            scoreSums = ArrayUtil.grow(scoreSums);
-            if (scoreMode == ScoreMode.Min) {
-              Arrays.fill(scoreSums, begin, scoreSums.length, Float.POSITIVE_INFINITY);
-            } else if (scoreMode == ScoreMode.Max) {
-              Arrays.fill(scoreSums, begin, scoreSums.length, Float.NEGATIVE_INFINITY);
+      if (doc > docValues.docID()) {
+        docValues.advance(doc);
+      }
+      if (doc == docValues.docID()) {
+        long ord;
+        while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+          int termID = collectedTerms.add(docValues.lookupOrd(ord));
+          if (termID < 0) {
+            termID = -termID - 1;
+          } else {
+            if (termID >= scoreSums.length) {
+              int begin = scoreSums.length;
+              scoreSums = ArrayUtil.grow(scoreSums);
+              if (scoreMode == ScoreMode.Min) {
+                Arrays.fill(scoreSums, begin, scoreSums.length, Float.POSITIVE_INFINITY);
+              } else if (scoreMode == ScoreMode.Max) {
+                Arrays.fill(scoreSums, begin, scoreSums.length, Float.NEGATIVE_INFINITY);
+              }
             }
           }
-        }
         
-        switch (scoreMode) {
+          switch (scoreMode) {
           case Total:
             scoreSums[termID] += scorer.score();
             break;
@@ -218,6 +240,7 @@ abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV>
             break;
           default:
             throw new AssertionError("unexpected: " + scoreMode);
+          }
         }
       }
     }
@@ -232,21 +255,25 @@ abstract class TermsWithScoreCollector<DV> extends DocValuesTermsCollector<DV>
 
       @Override
       public void collect(int doc) throws IOException {
-        docValues.setDocument(doc);
-        long ord;
-        while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-          int termID = collectedTerms.add(docValues.lookupOrd(ord));
-          if (termID < 0) {
-            termID = -termID - 1;
-          } else {
-            if (termID >= scoreSums.length) {
-              scoreSums = ArrayUtil.grow(scoreSums);
-              scoreCounts = ArrayUtil.grow(scoreCounts);
+        if (doc > docValues.docID()) {
+          docValues.advance(doc);
+        }
+        if (doc == docValues.docID()) {
+          long ord;
+          while ((ord = docValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+            int termID = collectedTerms.add(docValues.lookupOrd(ord));
+            if (termID < 0) {
+              termID = -termID - 1;
+            } else {
+              if (termID >= scoreSums.length) {
+                scoreSums = ArrayUtil.grow(scoreSums);
+                scoreCounts = ArrayUtil.grow(scoreCounts);
+              }
             }
-          }
           
-          scoreSums[termID] += scorer.score();
-          scoreCounts[termID]++;
+            scoreSums[termID] += scorer.score();
+            scoreCounts[termID]++;
+          }
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
index d3ff410..0777150 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinSortField.java
@@ -16,7 +16,10 @@
  */
 package org.apache.lucene.search.join;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FilterNumericDocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
@@ -25,11 +28,8 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.NumericUtils;
 
-import java.io.IOException;
-
 /**
  * A special sort field that allows sorting parent docs based on nested / child level fields.
  * Based on the sort order it either takes the document with the lowest or highest field value into account.
@@ -139,16 +139,6 @@ public class ToParentBlockJoinSortField extends SortField {
         }
         return BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
       }
-      @Override
-      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
-        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitSet parents = parentFilter.getBitSet(context);
-        final BitSet children = childFilter.getBitSet(context);
-        if (children == null) {
-          return new Bits.MatchNoBits(context.reader().maxDoc());
-        }
-        return BlockJoinSelector.wrap(docsWithValue, parents, children);
-      }
     };
   }
 
@@ -167,16 +157,6 @@ public class ToParentBlockJoinSortField extends SortField {
         }
         return BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
       }
-      @Override
-      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
-        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitSet parents = parentFilter.getBitSet(context);
-        final BitSet children = childFilter.getBitSet(context);
-        if (children == null) {
-          return new Bits.MatchNoBits(context.reader().maxDoc());
-        }
-        return BlockJoinSelector.wrap(docsWithValue, parents, children);
-      }
     };
   }
 
@@ -193,12 +173,11 @@ public class ToParentBlockJoinSortField extends SortField {
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
-        // undo the numericutils sortability
-        return new NumericDocValues() {
+        return new FilterNumericDocValues(BlockJoinSelector.wrap(sortedNumeric, type, parents, children)) {
           @Override
-          public long get(int docID) {
-            return NumericUtils.sortableFloatBits((int) view.get(docID));
+          public long longValue() {
+            // undo the numericutils sortability
+            return NumericUtils.sortableFloatBits((int) super.longValue());
           }
         };
       }
@@ -218,25 +197,14 @@ public class ToParentBlockJoinSortField extends SortField {
         if (children == null) {
           return DocValues.emptyNumeric();
         }
-        final NumericDocValues view = BlockJoinSelector.wrap(sortedNumeric, type, parents, children);
-        // undo the numericutils sortability
-        return new NumericDocValues() {
+        return new FilterNumericDocValues(BlockJoinSelector.wrap(sortedNumeric, type, parents, children)) {
           @Override
-          public long get(int docID) {
-            return NumericUtils.sortableDoubleBits(view.get(docID));
+          public long longValue() {
+            // undo the numericutils sortability
+            return NumericUtils.sortableDoubleBits(super.longValue());
           }
         };
       }
-      @Override
-      protected Bits getDocsWithValue(LeafReaderContext context, String field) throws IOException {
-        final Bits docsWithValue = DocValues.getDocsWithField(context.reader(), field);
-        final BitSet parents = parentFilter.getBitSet(context);
-        final BitSet children = childFilter.getBitSet(context);
-        if (children == null) {
-          return new Bits.MatchNoBits(context.reader().maxDoc());
-        }
-        return BlockJoinSelector.wrap(docsWithValue, parents, children);
-      }
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
index c4cdc3b..41f994c 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSelector.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.search.join;
 
+import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.index.DocValues;
@@ -27,6 +28,8 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 public class TestBlockJoinSelector extends LuceneTestCase {
 
   public void testDocsWithValue() {
@@ -61,7 +64,7 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     assertFalse(docsWithValue.get(19));
   }
 
-  public void testSortedSelector() {
+  public void testSortedSelector() throws IOException {
     final BitSet parents = new FixedBitSet(20);
     parents.set(0);
     parents.set(5);
@@ -84,43 +87,91 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     ords[4] = 3;
     ords[12] = 10;
     ords[18] = 10;
-    final SortedDocValues values = new SortedDocValues() {
 
-      @Override
-      public int getOrd(int docID) {
-        return ords[docID];
-      }
+    final SortedDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(new CannedSortedDocValues(ords)), BlockJoinSelector.Type.MIN, parents, children);
+    assertEquals(5, mins.nextDoc());
+    assertEquals(3, mins.ordValue());
+    assertEquals(15, mins.nextDoc());
+    assertEquals(10, mins.ordValue());
+    assertEquals(19, mins.nextDoc());
+    assertEquals(10, mins.ordValue());
+    assertEquals(NO_MORE_DOCS, mins.nextDoc());
+
+    final SortedDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(new CannedSortedDocValues(ords)), BlockJoinSelector.Type.MAX, parents, children);
+    assertEquals(5, maxs.nextDoc());
+    assertEquals(7, maxs.ordValue());
+    assertEquals(15, maxs.nextDoc());
+    assertEquals(10, maxs.ordValue());
+    assertEquals(19, maxs.nextDoc());
+    assertEquals(10, maxs.ordValue());
+    assertEquals(NO_MORE_DOCS, maxs.nextDoc());
+  }
+
+  private static class CannedSortedDocValues extends SortedDocValues {
+    private final int[] ords;
+    int docID = -1;
 
-      @Override
-      public BytesRef lookupOrd(int ord) {
-        throw new UnsupportedOperationException();
+    public CannedSortedDocValues(int[] ords) {
+      this.ords = ords;
+    }
+
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (true) {
+        docID++;
+        if (docID == ords.length) {
+          docID = NO_MORE_DOCS;
+          break;
+        }
+        if (ords[docID] != -1) {
+          break;
+        }
       }
+      return docID;
+    }
 
-      @Override
-      public int getValueCount() {
-        return 11;
+    @Override
+    public int advance(int target) {
+      if (target >= ords.length) {
+        docID = NO_MORE_DOCS;
+      } else {
+        docID = target;
+        if (ords[docID] == -1) {
+          nextDoc();
+        }
       }
+      return docID;
+    }
+
+    @Override
+    public int ordValue() {
+      assert ords[docID] != -1;
+      return ords[docID];
+    }
+
+    @Override
+    public long cost() {
+      return 5;
+    }
+        
+    @Override
+    public BytesRef lookupOrd(int ord) {
+      throw new UnsupportedOperationException();
+    }
 
-    };
-
-    final SortedDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(values), BlockJoinSelector.Type.MIN, parents, children);
-    assertEquals(-1, mins.getOrd(0));
-    assertEquals(3, mins.getOrd(5));
-    assertEquals(-1, mins.getOrd(6));
-    assertEquals(-1, mins.getOrd(10));
-    assertEquals(10, mins.getOrd(15));
-    assertEquals(-1, mins.getOrd(19));
-
-    final SortedDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(values), BlockJoinSelector.Type.MAX, parents, children);
-    assertEquals(-1, maxs.getOrd(0));
-    assertEquals(7, maxs.getOrd(5));
-    assertEquals(-1, maxs.getOrd(6));
-    assertEquals(-1, maxs.getOrd(10));
-    assertEquals(10, maxs.getOrd(15));
-    assertEquals(-1, maxs.getOrd(19));
+    @Override
+    public int getValueCount() {
+      return 11;
+    }
   }
 
-  public void testNumericSelector() {
+  public void testNumericSelector() throws Exception {
     final BitSet parents = new FixedBitSet(20);
     parents.set(0);
     parents.set(5);
@@ -148,29 +199,71 @@ public class TestBlockJoinSelector extends LuceneTestCase {
     longs[12] = 10;
     docsWithValue.set(18);
     longs[18] = 10;
-    final NumericDocValues values = new NumericDocValues() {
 
-      @Override
-      public long get(int docID) {
-        return longs[docID];
+    final NumericDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(new CannedNumericDocValues(longs, docsWithValue)), BlockJoinSelector.Type.MIN, parents, children);
+    assertEquals(5, mins.nextDoc());
+    assertEquals(3, mins.longValue());
+    assertEquals(15, mins.nextDoc());
+    assertEquals(10, mins.longValue());
+    assertEquals(NO_MORE_DOCS, mins.nextDoc());
+
+    final NumericDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(new CannedNumericDocValues(longs, docsWithValue)), BlockJoinSelector.Type.MAX, parents, children);
+    assertEquals(5, maxs.nextDoc());
+    assertEquals(7, maxs.longValue());
+    assertEquals(15, maxs.nextDoc());
+    assertEquals(10, maxs.longValue());
+    assertEquals(NO_MORE_DOCS, maxs.nextDoc());
+  }
+
+  private static class CannedNumericDocValues extends NumericDocValues {
+    final Bits docsWithValue;
+    final long[] values;
+    int docID = -1;
+
+    public CannedNumericDocValues(long[] values, Bits docsWithValue) {
+      this.values = values;
+      this.docsWithValue = docsWithValue;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (true) {
+        docID++;
+        if (docID == values.length) {
+          docID = NO_MORE_DOCS;
+          break;
+        }
+        if (docsWithValue.get(docID)) {
+          break;
+        }
+      }
+      return docID;
+    }
+
+    @Override
+    public int advance(int target) {
+      if (target >= values.length) {
+        docID = NO_MORE_DOCS;
+        return docID;
+      } else {
+        docID = target - 1;
+        return nextDoc();
       }
-      
-    };
-
-    final NumericDocValues mins = BlockJoinSelector.wrap(DocValues.singleton(values, docsWithValue), BlockJoinSelector.Type.MIN, parents, children);
-    assertEquals(0, mins.get(0));
-    assertEquals(3, mins.get(5));
-    assertEquals(0, mins.get(6));
-    assertEquals(0, mins.get(10));
-    assertEquals(10, mins.get(15));
-    assertEquals(0, mins.get(19));
-
-    final NumericDocValues maxs = BlockJoinSelector.wrap(DocValues.singleton(values, docsWithValue), BlockJoinSelector.Type.MAX, parents, children);
-    assertEquals(0, maxs.get(0));
-    assertEquals(7, maxs.get(5));
-    assertEquals(0, maxs.get(6));
-    assertEquals(0, maxs.get(10));
-    assertEquals(10, maxs.get(15));
-    assertEquals(0, maxs.get(19));
+    }
+
+    @Override
+    public long longValue() {
+      return values[docID];
+    }
+
+    @Override
+    public long cost() {
+      return 5;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
index 8b2a0bd..0d53c85 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoinSorting.java
@@ -16,13 +16,15 @@
  */
 package org.apache.lucene.search.join;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.NoMergePolicy;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FieldDoc;
@@ -37,9 +39,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  */
 public class TestBlockJoinSorting extends LuceneTestCase {
@@ -48,7 +47,7 @@ public class TestBlockJoinSorting extends LuceneTestCase {
   public void testNestedSorting() throws Exception {
     final Directory dir = newDirectory();
     final RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig(new MockAnalyzer(random()))
-        .setMergePolicy(NoMergePolicy.INSTANCE));
+                                                      .setMergePolicy(newLogMergePolicy()));
 
     List<Document> docs = new ArrayList<>();
     Document document = new Document();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
index 17cd4a2..2bd8381 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
@@ -54,8 +54,8 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
+import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.NoMergePolicy;
 import org.apache.lucene.index.NumericDocValues;
@@ -88,7 +88,6 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.BitSetIterator;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
@@ -484,7 +483,8 @@ public class TestJoinUtil extends LuceneTestCase {
             return new FilterScorer(fieldScorer, this) {
               @Override
               public float score() throws IOException {
-                return (float) price.get(in.docID());
+                assertEquals(in.docID(), price.nextDoc());
+                return (float) price.longValue();
               }
             };
           }
@@ -1182,15 +1182,19 @@ public class TestJoinUtil extends LuceneTestCase {
 
           @Override
           public void collect(int doc) throws IOException {
-            docTermOrds.setDocument(doc);
-            long ord;
-            while ((ord = docTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-              final BytesRef joinValue = docTermOrds.lookupOrd(ord);
-              JoinScore joinScore = joinValueToJoinScores.get(joinValue);
-              if (joinScore == null) {
-                joinValueToJoinScores.put(BytesRef.deepCopyOf(joinValue), joinScore = new JoinScore());
+            if (doc > docTermOrds.docID()) {
+              docTermOrds.advance(doc);
+            }
+            if (doc == docTermOrds.docID()) {
+              long ord;
+              while ((ord = docTermOrds.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+                final BytesRef joinValue = docTermOrds.lookupOrd(ord);
+                JoinScore joinScore = joinValueToJoinScores.get(joinValue);
+                if (joinScore == null) {
+                  joinValueToJoinScores.put(BytesRef.deepCopyOf(joinValue), joinScore = new JoinScore());
+                }
+                joinScore.addScore(scorer.score());
               }
-              joinScore.addScore(scorer.score());
             }
           }
 
@@ -1214,12 +1218,17 @@ public class TestJoinUtil extends LuceneTestCase {
 
           private Scorer scorer;
           private BinaryDocValues terms;
-          private Bits docsWithField;
 
           @Override
           public void collect(int doc) throws IOException {
-            final BytesRef joinValue = terms.get(doc);
-            if (joinValue.length == 0 && !docsWithField.get(doc)) {
+            if (doc > terms.docID()) {
+              terms.advance(doc);
+            }
+            final BytesRef joinValue;
+            if (doc == terms.docID()) {
+              joinValue = terms.binaryValue();
+            } else {
+              // missing;
               return;
             }
 
@@ -1236,7 +1245,6 @@ public class TestJoinUtil extends LuceneTestCase {
           @Override
           protected void doSetNextReader(LeafReaderContext context) throws IOException {
             terms = DocValues.getBinary(context.reader(), fromField);
-            docsWithField = DocValues.getDocsWithField(context.reader(), fromField);
           }
 
           @Override
@@ -1281,8 +1289,17 @@ public class TestJoinUtil extends LuceneTestCase {
           private int docBase;
 
           @Override
-          public void collect(int doc) {
-            final BytesRef joinValue = terms.get(doc);
+          public void collect(int doc) throws IOException {
+            if (doc > terms.docID()) {
+              terms.advance(doc);
+            }
+            final BytesRef joinValue;
+            if (doc == terms.docID()) {
+              joinValue = terms.binaryValue();
+            } else {
+              // missing;
+              joinValue = new BytesRef(BytesRef.EMPTY_BYTES);
+            }
             JoinScore joinScore = joinValueToJoinScores.get(joinValue);
             if (joinScore == null) {
               return;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 9037dfa..8db9c9e 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -828,7 +828,7 @@ public class MemoryIndex {
     private FieldInfo fieldInfo;
 
     /** The norms for this field; computed on demand. */
-    private transient NumericDocValues norms;
+    private transient LegacyNumericDocValues norms;
 
     /**
      * Term strings and their positions for this field: Map &lt;String
@@ -937,24 +937,47 @@ public class MemoryIndex {
     }
 
     NumericDocValues getNormDocValues() {
-      if (norms == null) {
-        FieldInvertState invertState = new FieldInvertState(fieldInfo.name, fieldInfo.number,
-            numTokens, numOverlapTokens, 0, boost);
-        final long value = normSimilarity.computeNorm(invertState);
-        if (DEBUG) System.err.println("MemoryIndexReader.norms: " + fieldInfo.name + ":" + value + ":" + numTokens);
-        norms = new NumericDocValues() {
+      FieldInvertState invertState = new FieldInvertState(fieldInfo.name, fieldInfo.number,
+                                                          numTokens, numOverlapTokens, 0, boost);
+      final long value = normSimilarity.computeNorm(invertState);
+      if (DEBUG) System.err.println("MemoryIndexReader.norms: " + fieldInfo.name + ":" + value + ":" + numTokens);
+      return new NumericDocValues() {
+          private int docID = -1;
 
           @Override
-          public long get(int docID) {
-            if (docID != 0)
-              throw new IndexOutOfBoundsException();
-            else
-              return value;
+          public int nextDoc() {
+            docID++;
+            if (docID == 1) {
+              docID = NO_MORE_DOCS;
+            }
+            return docID;
+          }
+
+          @Override
+          public int docID() {
+            return docID;
           }
 
+          @Override
+          public int advance(int target) {
+            if (docID <= 0 && target == 0) {
+              docID = 0;
+            } else {
+              docID = NO_MORE_DOCS;
+            }
+            return docID;
+          }
+
+          @Override
+          public long cost() {
+            return 1;
+          }
+
+          @Override
+          public long longValue() {
+            return value;
+          }
         };
-      }
-      return norms;
     }
   }
   
@@ -965,13 +988,13 @@ public class MemoryIndex {
   private static final class BinaryDocValuesProducer {
 
     BytesRefHash dvBytesValuesSet;
-    final SortedDocValues sortedDocValues;
+    final LegacySortedDocValues sortedDocValues;
     final BytesRef spare = new BytesRef();
 
     int[] bytesIds;
 
     private BinaryDocValuesProducer() {
-      sortedDocValues = new SortedDocValues() {
+      sortedDocValues = new LegacySortedDocValues() {
         @Override
         public int getOrd(int docID) {
           return 0;
@@ -1004,17 +1027,17 @@ public class MemoryIndex {
     long[] dvLongValues;
     int count;
 
-    final NumericDocValues numericDocValues;
-    final SortedNumericDocValues sortedNumericDocValues;
+    final LegacyNumericDocValues numericDocValues;
+    final LegacySortedNumericDocValues sortedNumericDocValues;
 
     private NumericDocValuesProducer() {
-      this.numericDocValues = new NumericDocValues() {
+      this.numericDocValues = new LegacyNumericDocValues() {
         @Override
         public long get(int docID) {
           return dvLongValues[0];
         }
       };
-      this.sortedNumericDocValues = new SortedNumericDocValues() {
+      this.sortedNumericDocValues = new LegacySortedNumericDocValues() {
         @Override
         public void setDocument(int doc) {
         }
@@ -1101,13 +1124,12 @@ public class MemoryIndex {
     }
 
     @Override
-    public NumericDocValues getNumericDocValues(String field) {
+    public NumericDocValues getNumericDocValues(String field) throws IOException {
       Info info = getInfoForExpectedDocValuesType(field, DocValuesType.NUMERIC);
-      if (info != null) {
-        return info.numericProducer.numericDocValues;
-      } else {
+      if (info == null) {
         return null;
       }
+      return new LegacyNumericDocValuesWrapper(new Bits.MatchAllBits(1), info.numericProducer.numericDocValues);
     }
 
     @Override
@@ -1123,7 +1145,7 @@ public class MemoryIndex {
     private SortedDocValues getSortedDocValues(String field, DocValuesType docValuesType) {
       Info info = getInfoForExpectedDocValuesType(field, docValuesType);
       if (info != null) {
-        return info.binaryProducer.sortedDocValues;
+        return new LegacySortedDocValuesWrapper(info.binaryProducer.sortedDocValues, 1);
       } else {
         return null;
       }
@@ -1133,7 +1155,7 @@ public class MemoryIndex {
     public SortedNumericDocValues getSortedNumericDocValues(String field) {
       Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_NUMERIC);
       if (info != null) {
-        return info.numericProducer.sortedNumericDocValues;
+        return new LegacySortedNumericDocValuesWrapper(info.numericProducer.sortedNumericDocValues, 1);
       } else {
         return null;
       }
@@ -1143,7 +1165,7 @@ public class MemoryIndex {
     public SortedSetDocValues getSortedSetDocValues(String field) {
       Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_SET);
       if (info != null) {
-        return new SortedSetDocValues() {
+        return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
 
           int index = 0;
 
@@ -1169,17 +1191,7 @@ public class MemoryIndex {
           public long getValueCount() {
             return info.binaryProducer.dvBytesValuesSet.size();
           }
-        };
-      } else {
-        return null;
-      }
-    }
-
-    @Override
-    public Bits getDocsWithField(String field) throws IOException {
-      Info info = fields.get(field);
-      if (info != null && info.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
-        return new Bits.MatchAllBits(1);
+          }, 1);
       } else {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index 1d7c60e..da3dd4c 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -180,7 +180,9 @@ public class TestMemoryIndex extends LuceneTestCase {
 
     IndexSearcher searcher = mi.createSearcher();
     LeafReader reader = (LeafReader) searcher.getIndexReader();
-    float n1 = reader.getNormValues("f1").get(0);
+    NumericDocValues norms = reader.getNormValues("f1");
+    assertEquals(0, norms.nextDoc());
+    float n1 = norms.longValue();
 
     // Norms are re-computed when we change the Similarity
     mi.setSimilarity(new ClassicSimilarity() {
@@ -189,7 +191,9 @@ public class TestMemoryIndex extends LuceneTestCase {
         return 74;
       }
     });
-    float n2 = reader.getNormValues("f1").get(0);
+    norms = reader.getNormValues("f1");
+    assertEquals(0, norms.nextDoc());
+    float n2 = norms.longValue();
 
     assertTrue(n1 != n2);
     TestUtil.checkReader(reader);
@@ -235,24 +239,27 @@ public class TestMemoryIndex extends LuceneTestCase {
     MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer);
     LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
     NumericDocValues numericDocValues = leafReader.getNumericDocValues("numeric");
-    assertEquals(29L, numericDocValues.get(0));
+    assertEquals(0, numericDocValues.nextDoc());
+    assertEquals(29L, numericDocValues.longValue());
     SortedNumericDocValues sortedNumericDocValues = leafReader.getSortedNumericDocValues("sorted_numeric");
-    sortedNumericDocValues.setDocument(0);
-    assertEquals(5, sortedNumericDocValues.count());
-    assertEquals(30L, sortedNumericDocValues.valueAt(0));
-    assertEquals(31L, sortedNumericDocValues.valueAt(1));
-    assertEquals(32L, sortedNumericDocValues.valueAt(2));
-    assertEquals(32L, sortedNumericDocValues.valueAt(3));
-    assertEquals(33L, sortedNumericDocValues.valueAt(4));
+    assertEquals(0, sortedNumericDocValues.nextDoc());
+    assertEquals(5, sortedNumericDocValues.docValueCount());
+    assertEquals(30L, sortedNumericDocValues.nextValue());
+    assertEquals(31L, sortedNumericDocValues.nextValue());
+    assertEquals(32L, sortedNumericDocValues.nextValue());
+    assertEquals(32L, sortedNumericDocValues.nextValue());
+    assertEquals(33L, sortedNumericDocValues.nextValue());
     BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
-    assertEquals("a", binaryDocValues.get(0).utf8ToString());
+    assertEquals(0, binaryDocValues.nextDoc());
+    assertEquals("a", binaryDocValues.binaryValue().utf8ToString());
     SortedDocValues sortedDocValues = leafReader.getSortedDocValues("sorted");
-    assertEquals("b", sortedDocValues.get(0).utf8ToString());
-    assertEquals(0, sortedDocValues.getOrd(0));
+    assertEquals(0, sortedDocValues.nextDoc());
+    assertEquals("b", sortedDocValues.binaryValue().utf8ToString());
+    assertEquals(0, sortedDocValues.ordValue());
     assertEquals("b", sortedDocValues.lookupOrd(0).utf8ToString());
     SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
     assertEquals(3, sortedSetDocValues.getValueCount());
-    sortedSetDocValues.setDocument(0);
+    assertEquals(0, sortedSetDocValues.nextDoc());
     assertEquals(0L, sortedSetDocValues.nextOrd());
     assertEquals(1L, sortedSetDocValues.nextOrd());
     assertEquals(2L, sortedSetDocValues.nextOrd());
@@ -335,7 +342,8 @@ public class TestMemoryIndex extends LuceneTestCase {
     assertEquals(5, penum.endOffset());
 
     BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("text");
-    assertEquals("quick brown fox", binaryDocValues.get(0).utf8ToString());
+    assertEquals(0, binaryDocValues.nextDoc());
+    assertEquals("quick brown fox", binaryDocValues.binaryValue().utf8ToString());
   }
 
   public void testPointValues() throws Exception {
@@ -475,7 +483,9 @@ public class TestMemoryIndex extends LuceneTestCase {
     assertArrayEquals(packedPoint, leafReader.getPointValues().getMinPackedValue("field"));
     assertArrayEquals(packedPoint, leafReader.getPointValues().getMaxPackedValue("field"));
 
-    assertEquals("term", leafReader.getBinaryDocValues("field").get(0).utf8ToString());
+    BinaryDocValues dvs = leafReader.getBinaryDocValues("field");
+    assertEquals(0, dvs.nextDoc());
+    assertEquals("term", dvs.binaryValue().utf8ToString());
   }
 
   public void testToStringDebug() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
index a785720..03c17a5 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
@@ -52,26 +52,8 @@ import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.CompositeReader;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
@@ -201,7 +183,9 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
         if (normValues != null) {
           // mem idx always computes norms on the fly
           assertNotNull(memNormValues);
-          assertEquals(normValues.get(0), memNormValues.get(0));
+          assertEquals(0, normValues.nextDoc());
+          assertEquals(0, memNormValues.nextDoc());
+          assertEquals(normValues.longValue(), memNormValues.longValue());
         }
           
         assertNotNull(memTerms);
@@ -503,32 +487,38 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
 
     NumericDocValues numericDocValues = leafReader.getNumericDocValues("numeric");
     NumericDocValues controlNumericDocValues = controlLeafReader.getNumericDocValues("numeric");
-    assertEquals(controlNumericDocValues.get(0), numericDocValues.get(0));
+    assertEquals(0, numericDocValues.nextDoc());
+    assertEquals(0, controlNumericDocValues.nextDoc());
+    assertEquals(controlNumericDocValues.longValue(), numericDocValues.longValue());
 
     SortedNumericDocValues sortedNumericDocValues = leafReader.getSortedNumericDocValues("sorted_numeric");
-    sortedNumericDocValues.setDocument(0);
+    assertEquals(0, sortedNumericDocValues.nextDoc());
     SortedNumericDocValues controlSortedNumericDocValues = controlLeafReader.getSortedNumericDocValues("sorted_numeric");
-    controlSortedNumericDocValues.setDocument(0);
-    assertEquals(controlSortedNumericDocValues.count(), sortedNumericDocValues.count());
-    for (int i = 0; i < controlSortedNumericDocValues.count(); i++) {
-      assertEquals(controlSortedNumericDocValues.valueAt(i), sortedNumericDocValues.valueAt(i));
+    assertEquals(0, controlSortedNumericDocValues.nextDoc());
+    assertEquals(controlSortedNumericDocValues.docValueCount(), sortedNumericDocValues.docValueCount());
+    for (int i = 0; i < controlSortedNumericDocValues.docValueCount(); i++) {
+      assertEquals(controlSortedNumericDocValues.nextValue(), sortedNumericDocValues.nextValue());
     }
 
     BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
     BinaryDocValues controlBinaryDocValues = controlLeafReader.getBinaryDocValues("binary");
-    assertEquals(controlBinaryDocValues.get(0), binaryDocValues.get(0));
+    assertEquals(0, binaryDocValues.nextDoc());
+    assertEquals(0, controlBinaryDocValues.nextDoc());
+    assertEquals(controlBinaryDocValues.binaryValue(), binaryDocValues.binaryValue());
 
     SortedDocValues sortedDocValues = leafReader.getSortedDocValues("sorted");
     SortedDocValues controlSortedDocValues = controlLeafReader.getSortedDocValues("sorted");
     assertEquals(controlSortedDocValues.getValueCount(), sortedDocValues.getValueCount());
-    assertEquals(controlSortedDocValues.get(0), sortedDocValues.get(0));
-    assertEquals(controlSortedDocValues.getOrd(0), sortedDocValues.getOrd(0));
+    assertEquals(0, sortedDocValues.nextDoc());
+    assertEquals(0, controlSortedDocValues.nextDoc());
+    assertEquals(controlSortedDocValues.binaryValue(), sortedDocValues.binaryValue());
+    assertEquals(controlSortedDocValues.ordValue(), sortedDocValues.ordValue());
     assertEquals(controlSortedDocValues.lookupOrd(0), sortedDocValues.lookupOrd(0));
 
     SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
-    sortedSetDocValues.setDocument(0);
+    assertEquals(0, sortedSetDocValues.nextDoc());
     SortedSetDocValues controlSortedSetDocValues = controlLeafReader.getSortedSetDocValues("sorted_set");
-    controlSortedSetDocValues.setDocument(0);
+    assertEquals(0, controlSortedSetDocValues.nextDoc());
     assertEquals(controlSortedSetDocValues.getValueCount(), sortedSetDocValues.getValueCount());
     for (long controlOrd = controlSortedSetDocValues.nextOrd(); controlOrd != SortedSetDocValues.NO_MORE_ORDS;
          controlOrd = controlSortedSetDocValues.nextOrd()) {
@@ -563,7 +553,11 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
     IndexReader controlIndexReader = DirectoryReader.open(dir);
     LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
 
-    assertEquals(controlLeafReader.getNormValues("text").get(0), leafReader.getNormValues("text").get(0));
+    NumericDocValues norms = controlLeafReader.getNormValues("text");
+    assertEquals(0, norms.nextDoc());
+    NumericDocValues norms2 = leafReader.getNormValues("text");
+    assertEquals(0, norms2.nextDoc());
+    assertEquals(norms.longValue(), norms2.longValue());
 
     controlIndexReader.close();
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java b/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java
index f2016cc..6b75622 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java
@@ -110,7 +110,7 @@ public abstract class DiversifiedTopDocsCollector extends
   }
 
   protected ScoreDocKey insert(ScoreDocKey addition, int docBase,
-      NumericDocValues keys) {
+      NumericDocValues keys) throws IOException {
     if ((globalQueue.size() >= numHits)
         && (globalQueue.lessThan(addition, globalQueue.top()))) {
       // Queue is full and proposed addition is not a globally
@@ -122,7 +122,17 @@ public abstract class DiversifiedTopDocsCollector extends
     // We delay fetching the key until we are certain the score is globally
     // competitive. We need to adjust the ScoreDoc's global doc value to be
     // a leaf reader value when looking up keys
-    addition.key = keys.get(addition.doc - docBase);
+    int leafDocID = addition.doc - docBase;
+    long value;
+    if (keys.docID() < leafDocID) {
+      keys.advance(leafDocID);
+    }
+    if (keys.docID() == leafDocID) {
+      value = keys.longValue();
+    } else {
+      value = 0;
+    }
+    addition.key = value;
 
     // For this to work the choice of key class needs to implement
     // hashcode and equals.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
index 05a3b23..185b7ab 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
@@ -127,11 +127,32 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     protected NumericDocValues getKeys(final LeafReaderContext context) {
 
       return new NumericDocValues() {
+
+        @Override
+        public int docID() {
+          return sdv.docID() - context.docBase;
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          return sdv.nextDoc() - context.docBase;
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          return sdv.advance(target + context.docBase);
+        }
+
+        @Override
+        public long cost() {
+          return 0;
+        }
+        
         @Override
-        public long get(int docID) {
+        public long longValue() {
           // Keys are always expressed as a long so we obtain the
           // ordinal for our String-based artist name here
-          return sdv.getOrd(context.docBase + docID);
+          return sdv.ordValue();
         }
       };
     }
@@ -139,8 +160,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
 
   // Alternative, faster implementation for converting String keys to longs
   // but with the potential for hash collisions
-  private static final class HashedDocValuesDiversifiedCollector extends
-      DiversifiedTopDocsCollector {
+  private static final class HashedDocValuesDiversifiedCollector extends DiversifiedTopDocsCollector {
 
     private final String field;
     private BinaryDocValues vals;
@@ -155,8 +175,24 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     protected NumericDocValues getKeys(LeafReaderContext context) {
       return new NumericDocValues() {
         @Override
-        public long get(int docID) {
-          return vals == null ? -1 : vals.get(docID).hashCode();
+        public int docID() {
+          return vals.docID();
+        }
+        @Override
+        public int nextDoc() throws IOException {
+          return vals.nextDoc();
+        }
+        @Override
+        public int advance(int target) throws IOException {
+          return vals.advance(target);
+        }
+        @Override
+        public long cost() {
+          return vals.cost();
+        }
+        @Override
+        public long longValue() {
+          return vals == null ? -1 : vals.binaryValue().hashCode();
         }
       };
     }
@@ -277,7 +313,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
 
   private DiversifiedTopDocsCollector doDiversifiedSearch(int numResults,
       int maxResultsPerArtist) throws IOException {
-    // Alternate between implementations used for key lookups 
+    // Alternate between implementations used for key lookups
     if (random().nextBoolean()) {
       // Faster key lookup but with potential for collisions on larger datasets
       return doFuzzyDiversifiedSearch(numResults, maxResultsPerArtist);
@@ -427,13 +463,19 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     public SimScorer simScorer(SimWeight stats, LeafReaderContext context)
         throws IOException {
       final SimScorer sub = sim.simScorer(stats, context);
-      final NumericDocValues values = DocValues.getNumeric(context.reader(),
-          scoreValueField);
+      final NumericDocValues values = DocValues.getNumeric(context.reader(), scoreValueField);
 
       return new SimScorer() {
         @Override
-        public float score(int doc, float freq) {
-          return Float.intBitsToFloat((int) values.get(doc));
+        public float score(int doc, float freq) throws IOException {
+          if (doc != values.docID()) {
+            values.advance(doc);
+          }
+          if (doc == values.docID()) {
+            return Float.intBitsToFloat((int) values.longValue());
+          } else {
+            return 0f;
+          }
         }
 
         @Override
@@ -448,12 +490,10 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
         }
 
         @Override
-        public Explanation explain(int doc, Explanation freq) {
-          return Explanation.match(Float.intBitsToFloat((int) values.get(doc)),
-              "indexDocValue(" + scoreValueField + ")");
+        public Explanation explain(int doc, Explanation freq) throws IOException {
+          return Explanation.match(score(doc, 0f), "indexDocValue(" + scoreValueField + ")");
         }
       };
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionValues.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionValues.java b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionValues.java
index 1e7590d..9e73b4b 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionValues.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionValues.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.queries.function;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Scorer;
@@ -38,22 +40,22 @@ import org.apache.lucene.util.mutable.MutableValueFloat;
 //   want the Query carrying around big objects
 public abstract class FunctionValues {
 
-  public byte byteVal(int doc) { throw new UnsupportedOperationException(); }
-  public short shortVal(int doc) { throw new UnsupportedOperationException(); }
+  public byte byteVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
+  public short shortVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
 
-  public float floatVal(int doc) { throw new UnsupportedOperationException(); }
-  public int intVal(int doc) { throw new UnsupportedOperationException(); }
-  public long longVal(int doc) { throw new UnsupportedOperationException(); }
-  public double doubleVal(int doc) { throw new UnsupportedOperationException(); }
+  public float floatVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
+  public int intVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
+  public long longVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
+  public double doubleVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
   // TODO: should we make a termVal, returns BytesRef?
-  public String strVal(int doc) { throw new UnsupportedOperationException(); }
+  public String strVal(int doc) throws IOException { throw new UnsupportedOperationException(); }
 
-  public boolean boolVal(int doc) {
+  public boolean boolVal(int doc) throws IOException {
     return intVal(doc) != 0;
   }
 
   /** returns the bytes representation of the string val - TODO: should this return the indexed raw bytes not? */
-  public boolean bytesVal(int doc, BytesRefBuilder target) {
+  public boolean bytesVal(int doc, BytesRefBuilder target) throws IOException {
     String s = strVal(doc);
     if (s==null) {
       target.clear();
@@ -64,13 +66,13 @@ public abstract class FunctionValues {
   }
 
   /** Native Java Object representation of the value */
-  public Object objectVal(int doc) {
+  public Object objectVal(int doc) throws IOException {
     // most FunctionValues are functions, so by default return a Float()
     return floatVal(doc);
   }
 
   /** Returns true if there is a value for this document */
-  public boolean exists(int doc) {
+  public boolean exists(int doc) throws IOException {
     return true;
   }
 
@@ -79,13 +81,15 @@ public abstract class FunctionValues {
    * @return the sort ordinal for the specified doc
    * TODO: Maybe we can just use intVal for this...
    */
-  public int ordVal(int doc) { throw new UnsupportedOperationException(); }
+  public int ordVal(int doc) throws IOException {
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * @return the number of unique sort ordinals this instance has
    */
   public int numOrd() { throw new UnsupportedOperationException(); }
-  public abstract String toString(int doc);
+  public abstract String toString(int doc) throws IOException;
 
   /**
    * Abstraction of the logic required to fill the value of a specified doc into
@@ -100,7 +104,7 @@ public abstract class FunctionValues {
     public abstract MutableValue getValue();
 
     /** MutableValue will be reused across calls.  Returns true if the value exists. */
-    public abstract void fillValue(int doc);
+    public abstract void fillValue(int doc) throws IOException;
   }
 
   /** @lucene.experimental  */
@@ -114,25 +118,25 @@ public abstract class FunctionValues {
       }
 
       @Override
-      public void fillValue(int doc) {
+      public void fillValue(int doc) throws IOException {
         mval.value = floatVal(doc);
       }
     };
   }
 
   //For Functions that can work with multiple values from the same document.  This does not apply to all functions
-  public void byteVal(int doc, byte [] vals) { throw new UnsupportedOperationException(); }
-  public void shortVal(int doc, short [] vals) { throw new UnsupportedOperationException(); }
+  public void byteVal(int doc, byte [] vals) throws IOException { throw new UnsupportedOperationException(); }
+  public void shortVal(int doc, short [] vals) throws IOException { throw new UnsupportedOperationException(); }
 
-  public void floatVal(int doc, float [] vals) { throw new UnsupportedOperationException(); }
-  public void intVal(int doc, int [] vals) { throw new UnsupportedOperationException(); }
-  public void longVal(int doc, long [] vals) { throw new UnsupportedOperationException(); }
-  public void doubleVal(int doc, double [] vals) { throw new UnsupportedOperationException(); }
+  public void floatVal(int doc, float [] vals) throws IOException { throw new UnsupportedOperationException(); }
+  public void intVal(int doc, int [] vals) throws IOException { throw new UnsupportedOperationException(); }
+  public void longVal(int doc, long [] vals) throws IOException { throw new UnsupportedOperationException(); }
+  public void doubleVal(int doc, double [] vals) throws IOException { throw new UnsupportedOperationException(); }
 
   // TODO: should we make a termVal, fills BytesRef[]?
-  public void strVal(int doc, String [] vals) { throw new UnsupportedOperationException(); }
+  public void strVal(int doc, String [] vals) throws IOException { throw new UnsupportedOperationException(); }
 
-  public Explanation explain(int doc) {
+  public Explanation explain(int doc) throws IOException {
     return Explanation.match(floatVal(doc), toString(doc));
   }
 
@@ -157,7 +161,7 @@ public abstract class FunctionValues {
   // because it needs different behavior depending on the type of fields.  There is also
   // a setup cost - parsing and normalizing params, and doing a binary search on the StringIndex.
   // TODO: change "reader" to LeafReaderContext
-  public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
+  public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
     float lower;
     float upper;
 
@@ -178,7 +182,7 @@ public abstract class FunctionValues {
     if (includeLower && includeUpper) {
       return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           float docVal = floatVal(doc);
           return docVal >= l && docVal <= u;
@@ -188,7 +192,7 @@ public abstract class FunctionValues {
     else if (includeLower && !includeUpper) {
        return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           float docVal = floatVal(doc);
           return docVal >= l && docVal < u;
@@ -198,7 +202,7 @@ public abstract class FunctionValues {
     else if (!includeLower && includeUpper) {
        return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           float docVal = floatVal(doc);
           return docVal > l && docVal <= u;
@@ -208,7 +212,7 @@ public abstract class FunctionValues {
     else {
        return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           float docVal = floatVal(doc);
           return docVal > l && docVal < u;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
index 49d4b77..4064fc9 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
@@ -38,7 +38,9 @@ public abstract class ValueSource {
 
   /**
    * Gets the values for this reader and the context that was previously
-   * passed to createWeight()
+   * passed to createWeight().  The values must be consumed in a forward
+   * docID manner, and you must call this method again to iterate through
+   * the values again.
    */
   public abstract FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException;
 
@@ -144,12 +146,12 @@ public abstract class ValueSource {
     }
 
     @Override
-    public int compareBottom(int doc) {
+    public int compareBottom(int doc) throws IOException {
       return Double.compare(bottom, docVals.doubleVal(doc));
     }
 
     @Override
-    public void copy(int slot, int doc) {
+    public void copy(int slot, int doc) throws IOException {
       values[slot] = docVals.doubleVal(doc);
     }
 
@@ -174,7 +176,7 @@ public abstract class ValueSource {
     }
 
     @Override
-    public int compareTop(int doc) {
+    public int compareTop(int doc) throws IOException {
       final double docValue = docVals.doubleVal(doc);
       return Double.compare(topValue, docValue);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
index 035327b..c9303cd 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
@@ -62,7 +62,7 @@ public abstract class ValueSourceScorer extends Scorer {
   }
 
   /** Override to decide if this document matches. It's called by {@link TwoPhaseIterator#matches()}. */
-  public abstract boolean matches(int doc);
+  public abstract boolean matches(int doc) throws IOException;
 
   @Override
   public DocIdSetIterator iterator() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/BoolDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/BoolDocValues.java b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/BoolDocValues.java
index f3066e3..eacea1d 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/BoolDocValues.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/BoolDocValues.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.queries.function.docvalues;
 
+import java.io.IOException;
+
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.util.mutable.MutableValue;
@@ -33,50 +35,50 @@ public abstract class BoolDocValues extends FunctionValues {
   }
 
   @Override
-  public abstract boolean boolVal(int doc);
+  public abstract boolean boolVal(int doc) throws IOException;
 
   @Override
-  public byte byteVal(int doc) {
+  public byte byteVal(int doc) throws IOException {
     return boolVal(doc) ? (byte)1 : (byte)0;
   }
 
   @Override
-  public short shortVal(int doc) {
+  public short shortVal(int doc) throws IOException {
     return boolVal(doc) ? (short)1 : (short)0;
   }
 
   @Override
-  public float floatVal(int doc) {
+  public float floatVal(int doc) throws IOException {
     return boolVal(doc) ? (float)1 : (float)0;
   }
 
   @Override
-  public int intVal(int doc) {
+  public int intVal(int doc) throws IOException {
     return boolVal(doc) ? 1 : 0;
   }
 
   @Override
-  public long longVal(int doc) {
+  public long longVal(int doc) throws IOException {
     return boolVal(doc) ? (long)1 : (long)0;
   }
 
   @Override
-  public double doubleVal(int doc) {
+  public double doubleVal(int doc) throws IOException {
     return boolVal(doc) ? (double)1 : (double)0;
   }
 
   @Override
-  public String strVal(int doc) {
+  public String strVal(int doc) throws IOException {
     return Boolean.toString(boolVal(doc));
   }
 
   @Override
-  public Object objectVal(int doc) {
+  public Object objectVal(int doc) throws IOException {
     return exists(doc) ? boolVal(doc) : null;
   }
 
   @Override
-  public String toString(int doc) {
+  public String toString(int doc) throws IOException {
     return vs.description() + '=' + strVal(doc);
   }
 
@@ -91,7 +93,7 @@ public abstract class BoolDocValues extends FunctionValues {
       }
 
       @Override
-      public void fillValue(int doc) {
+      public void fillValue(int doc) throws IOException {
         mval.value = boolVal(doc);
         mval.exists = exists(doc);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DocTermsIndexDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DocTermsIndexDocValues.java b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DocTermsIndexDocValues.java
index 194969d..e855e27 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DocTermsIndexDocValues.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DocTermsIndexDocValues.java
@@ -39,26 +39,45 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
   protected final ValueSource vs;
   protected final MutableValueStr val = new MutableValueStr();
   protected final CharsRefBuilder spareChars = new CharsRefBuilder();
+  private final String field;
+  private int lastDocID;
 
   public DocTermsIndexDocValues(ValueSource vs, LeafReaderContext context, String field) throws IOException {
-    this(vs, open(context, field));
+    this(field, vs, open(context, field));
   }
   
-  protected DocTermsIndexDocValues(ValueSource vs, SortedDocValues termsIndex) {
+  protected DocTermsIndexDocValues(String field, ValueSource vs, SortedDocValues termsIndex) {
+    this.field = field;
     this.vs = vs;
     this.termsIndex = termsIndex;
   }
 
+  protected int getOrdForDoc(int doc) throws IOException {
+    if (doc < lastDocID) {
+      throw new IllegalArgumentException("docs were sent out-of-order: lastDocID=" + lastDocID + " vs docID=" + doc);
+    }
+    lastDocID = doc;
+    int curDocID = termsIndex.docID();
+    if (doc > curDocID) {
+      curDocID = termsIndex.advance(doc);
+    }
+    if (doc == curDocID) {
+      return termsIndex.ordValue();
+    } else {
+      return -1;
+    }
+  }
+
   protected abstract String toTerm(String readableValue);
 
   @Override
-  public boolean exists(int doc) {
-    return ordVal(doc) >= 0;
+  public boolean exists(int doc) throws IOException {
+    return getOrdForDoc(doc) >= 0;
   }
 
   @Override
-  public int ordVal(int doc) {
-    return termsIndex.getOrd(doc);
+  public int ordVal(int doc) throws IOException {
+    return getOrdForDoc(doc);
   }
 
   @Override
@@ -67,32 +86,36 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
   }
 
   @Override
-  public boolean bytesVal(int doc, BytesRefBuilder target) {
+  public boolean bytesVal(int doc, BytesRefBuilder target) throws IOException {
     target.clear();
-    target.copyBytes(termsIndex.get(doc));
-    return target.length() > 0;
+    if (getOrdForDoc(doc) == -1) {
+      return false;
+    } else {
+      target.copyBytes(termsIndex.binaryValue());
+      return true;
+    }
   }
 
   @Override
-  public String strVal(int doc) {
-    final BytesRef term = termsIndex.get(doc);
-    if (term.length == 0) {
+  public String strVal(int doc) throws IOException {
+    if (getOrdForDoc(doc) == -1) {
       return null;
     }
+    final BytesRef term = termsIndex.binaryValue();
     spareChars.copyUTF8Bytes(term);
     return spareChars.toString();
   }
 
   @Override
-  public boolean boolVal(int doc) {
+  public boolean boolVal(int doc) throws IOException {
     return exists(doc);
   }
 
   @Override
-  public abstract Object objectVal(int doc);  // force subclasses to override
+  public abstract Object objectVal(int doc) throws IOException;  // force subclasses to override
 
   @Override
-  public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
+  public ValueSourceScorer getRangeScorer(LeafReaderContext readerContext, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) throws IOException {
     // TODO: are lowerVal and upperVal in indexed form or not?
     lowerVal = lowerVal == null ? null : toTerm(lowerVal);
     upperVal = upperVal == null ? null : toTerm(upperVal);
@@ -121,16 +144,29 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
     final int uu = upper;
 
     return new ValueSourceScorer(readerContext, this) {
+      final SortedDocValues values = readerContext.reader().getSortedDocValues(field);
+      private int lastDocID;
+      
       @Override
-      public boolean matches(int doc) {
-        int ord = termsIndex.getOrd(doc);
-        return ord >= ll && ord <= uu;
+      public boolean matches(int doc) throws IOException {
+        if (doc < lastDocID) {
+          throw new IllegalArgumentException("docs were sent out-of-order: lastDocID=" + lastDocID + " vs docID=" + doc);
+        }
+        if (doc > values.docID()) {
+          values.advance(doc);
+        }
+        if (doc == values.docID()) {
+          int ord = values.ordValue();
+          return ord >= ll && ord <= uu;
+        } else {
+          return false;
+        }
       }
     };
   }
 
   @Override
-  public String toString(int doc) {
+  public String toString(int doc) throws IOException {
     return vs.description() + '=' + strVal(doc);
   }
 
@@ -145,8 +181,8 @@ public abstract class DocTermsIndexDocValues extends FunctionValues {
       }
 
       @Override
-      public void fillValue(int doc) {
-        int ord = termsIndex.getOrd(doc);
+      public void fillValue(int doc) throws IOException {
+        int ord = getOrdForDoc(doc);
         mval.value.clear();
         mval.exists = ord >= 0;
         if (mval.exists) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DoubleDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DoubleDocValues.java b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DoubleDocValues.java
index 91d1e6f..646db9c 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DoubleDocValues.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/docvalues/DoubleDocValues.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.queries.function.docvalues;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -35,50 +37,50 @@ public abstract class DoubleDocValues extends FunctionValues {
   }
 
   @Override
-  public byte byteVal(int doc) {
+  public byte byteVal(int doc) throws IOException {
     return (byte)doubleVal(doc);
   }
 
   @Override
-  public short shortVal(int doc) {
+  public short shortVal(int doc) throws IOException {
     return (short)doubleVal(doc);
   }
 
   @Override
-  public float floatVal(int doc) {
+  public float floatVal(int doc) throws IOException {
     return (float)doubleVal(doc);
   }
 
   @Override
-  public int intVal(int doc) {
+  public int intVal(int doc) throws IOException {
     return (int)doubleVal(doc);
   }
 
   @Override
-  public long longVal(int doc) {
+  public long longVal(int doc) throws IOException {
     return (long)doubleVal(doc);
   }
 
   @Override
-  public boolean boolVal(int doc) {
+  public boolean boolVal(int doc) throws IOException {
     return doubleVal(doc) != 0;
   }
 
   @Override
-  public abstract double doubleVal(int doc);
+  public abstract double doubleVal(int doc) throws IOException;
 
   @Override
-  public String strVal(int doc) {
+  public String strVal(int doc) throws IOException {
     return Double.toString(doubleVal(doc));
   }
 
   @Override
-  public Object objectVal(int doc) {
+  public Object objectVal(int doc) throws IOException {
     return exists(doc) ? doubleVal(doc) : null;
   }
 
   @Override
-  public String toString(int doc) {
+  public String toString(int doc) throws IOException {
     return vs.description() + '=' + strVal(doc);
   }
   
@@ -105,7 +107,7 @@ public abstract class DoubleDocValues extends FunctionValues {
     if (includeLower && includeUpper) {
       return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           double docVal = doubleVal(doc);
           return docVal >= l && docVal <= u;
@@ -115,7 +117,7 @@ public abstract class DoubleDocValues extends FunctionValues {
     else if (includeLower && !includeUpper) {
       return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           double docVal = doubleVal(doc);
           return docVal >= l && docVal < u;
@@ -125,7 +127,7 @@ public abstract class DoubleDocValues extends FunctionValues {
     else if (!includeLower && includeUpper) {
       return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           double docVal = doubleVal(doc);
           return docVal > l && docVal <= u;
@@ -135,7 +137,7 @@ public abstract class DoubleDocValues extends FunctionValues {
     else {
       return new ValueSourceScorer(readerContext, this) {
         @Override
-        public boolean matches(int doc) {
+        public boolean matches(int doc) throws IOException {
           if (!exists(doc)) return false;
           double docVal = doubleVal(doc);
           return docVal > l && docVal < u;
@@ -155,7 +157,7 @@ public abstract class DoubleDocValues extends FunctionValues {
       }
 
       @Override
-      public void fillValue(int doc) {
+      public void fillValue(int doc) throws IOException {
         mval.value = doubleVal(doc);
         mval.exists = exists(doc);
       }