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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValuesWrapper.java
new file mode 100644
index 0000000..13bc207
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValuesWrapper.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Wraps a {@link LegacyBinaryDocValues} into a {@link BinaryDocValues}.
+ *
+ * @deprecated Implement {@link BinaryDocValues} directly.
+ */
+@Deprecated
+public final class LegacyBinaryDocValuesWrapper extends BinaryDocValues {
+  private final Bits docsWithField;
+  private final LegacyBinaryDocValues values;
+  private final int maxDoc;
+  private int docID = -1;
+  
+  public LegacyBinaryDocValuesWrapper(Bits docsWithField, LegacyBinaryDocValues values) {
+    this.docsWithField = docsWithField;
+    this.values = values;
+    this.maxDoc = docsWithField.length();
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() {
+    docID++;
+    while (docID < maxDoc) {
+      if (docsWithField.get(docID)) {
+        return docID;
+      }
+      docID++;
+    }
+    docID = NO_MORE_DOCS;
+    return NO_MORE_DOCS;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target < docID) {
+      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
+    }
+    if (target == NO_MORE_DOCS) {
+      this.docID = NO_MORE_DOCS;
+    } else {
+      this.docID = target-1;
+      nextDoc();
+    }
+    return docID;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  @Override
+  public BytesRef binaryValue() {
+    return values.get(docID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValues.java
new file mode 100644
index 0000000..df71435
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValues.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.index;
+
+
+/**
+ * A per-document numeric value.
+ *
+ * @deprecated Use {@link NumericDocValues} instead.
+ */
+@Deprecated
+public abstract class LegacyNumericDocValues {
+  
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected LegacyNumericDocValues() {}
+
+  /**
+   * Returns the numeric value for the specified document ID.
+   * @param docID document ID to lookup
+   * @return numeric value
+   */
+  public abstract long get(int docID);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValuesWrapper.java
new file mode 100644
index 0000000..d9c997c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacyNumericDocValuesWrapper.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import org.apache.lucene.util.Bits;
+
+/**
+ * Wraps a {@link LegacyNumericDocValues} into a {@link NumericDocValues}.
+ *
+ * @deprecated Implement {@link NumericDocValues} directly.
+ */
+@Deprecated
+public final class LegacyNumericDocValuesWrapper extends NumericDocValues {
+  private final Bits docsWithField;
+  private final LegacyNumericDocValues values;
+  private final int maxDoc;
+  private int docID = -1;
+  
+  public LegacyNumericDocValuesWrapper(Bits docsWithField, LegacyNumericDocValues values) {
+    this.docsWithField = docsWithField;
+    this.values = values;
+    this.maxDoc = docsWithField.length();
+  }
+
+  /** Constructor used only for norms */
+  public LegacyNumericDocValuesWrapper(int maxDoc, LegacyNumericDocValues values) {
+    this(new Bits.MatchAllBits(maxDoc), values);
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() {
+    docID++;
+    while (docID < maxDoc) {
+      if (docsWithField.get(docID)) {
+        return docID;
+      }
+      docID++;
+    }
+    docID = NO_MORE_DOCS;
+    return NO_MORE_DOCS;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target < docID) {
+      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
+    }
+    if (target == NO_MORE_DOCS) {
+      this.docID = NO_MORE_DOCS;
+    } else {
+      this.docID = target-1;
+      nextDoc();
+    }
+    return docID;
+  }
+
+  @Override
+  public long cost() {
+    // TODO
+    return 0;
+  }
+
+  @Override
+  public long longValue() {
+    return values.get(docID);
+  }
+
+  @Override
+  public String toString() {
+    return "LegacyNumericDocValuesWrapper(" + values + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValues.java
new file mode 100644
index 0000000..089040f
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValues.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.index;
+
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A per-document byte[] with presorted values.
+ * <p>
+ * Per-Document values in a SortedDocValues are deduplicated, dereferenced,
+ * and sorted into a dictionary of unique values. A pointer to the
+ * dictionary value (ordinal) can be retrieved for each document. Ordinals
+ * are dense and in increasing sorted order.
+ *
+ * @deprecated Use {@link SortedDocValues} instead.
+ */
+@Deprecated
+public abstract class LegacySortedDocValues extends LegacyBinaryDocValues {
+
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected LegacySortedDocValues() {}
+
+  /**
+   * Returns the ordinal for the specified docID.
+   * @param  docID document ID to lookup
+   * @return ordinal for the document: this is dense, starts at 0, then
+   *         increments by 1 for the next value in sorted order. Note that
+   *         missing values are indicated by -1.
+   */
+  public abstract int getOrd(int docID);
+
+  /** Retrieves the value for the specified ordinal. The returned
+   * {@link BytesRef} may be re-used across calls to {@link #lookupOrd(int)}
+   * so make sure to {@link BytesRef#deepCopyOf(BytesRef) copy it} if you want
+   * to keep it around.
+   * @param ord ordinal to lookup (must be &gt;= 0 and &lt; {@link #getValueCount()})
+   * @see #getOrd(int) 
+   */
+  public abstract BytesRef lookupOrd(int ord);
+
+  /**
+   * Returns the number of unique values.
+   * @return number of unique values in this SortedDocValues. This is
+   *         also equivalent to one plus the maximum ordinal.
+   */
+  public abstract int getValueCount();
+
+  private final BytesRef empty = new BytesRef();
+
+  @Override
+  public BytesRef get(int docID) {
+    int ord = getOrd(docID);
+    if (ord == -1) {
+      return empty;
+    } else {
+      return lookupOrd(ord);
+    }
+  }
+
+  /** If {@code key} exists, returns its ordinal, else
+   *  returns {@code -insertionPoint-1}, like {@code
+   *  Arrays.binarySearch}.
+   *
+   *  @param key Key to look up
+   **/
+  public int lookupTerm(BytesRef key) {
+    int low = 0;
+    int high = getValueCount()-1;
+
+    while (low <= high) {
+      int mid = (low + high) >>> 1;
+      final BytesRef term = lookupOrd(mid);
+      int cmp = term.compareTo(key);
+
+      if (cmp < 0) {
+        low = mid + 1;
+      } else if (cmp > 0) {
+        high = mid - 1;
+      } else {
+        return mid; // key found
+      }
+    }
+
+    return -(low + 1);  // key not found.
+  }
+  
+  /** 
+   * Returns a {@link TermsEnum} over the values.
+   * The enum supports {@link TermsEnum#ord()} and {@link TermsEnum#seekExact(long)}.
+   */
+  public TermsEnum termsEnum() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValuesWrapper.java
new file mode 100644
index 0000000..d8ef2f4
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedDocValuesWrapper.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Wraps a {@link LegacySortedDocValues} into a {@link SortedDocValues}.
+ *
+ * @deprecated Implement {@link SortedDocValues} directly.
+ */
+@Deprecated
+public final class LegacySortedDocValuesWrapper extends SortedDocValues {
+  private final LegacySortedDocValues values;
+  private final int maxDoc;
+  private int docID = -1;
+  private int ord;
+  
+  public LegacySortedDocValuesWrapper(LegacySortedDocValues values, int maxDoc) {
+    this.values = values;
+    this.maxDoc = maxDoc;
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() {
+    assert docID != NO_MORE_DOCS;
+    docID++;
+    while (docID < maxDoc) {
+      ord = values.getOrd(docID);
+      if (ord != -1) {
+        return docID;
+      }
+      docID++;
+    }
+    docID = NO_MORE_DOCS;
+    return NO_MORE_DOCS;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target < docID) {
+      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
+    }
+    if (target >= maxDoc) {
+      this.docID = NO_MORE_DOCS;
+    } else {
+      this.docID = target-1;
+      nextDoc();
+    }
+    return docID;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  @Override
+  public int ordValue() {
+    return ord;
+  }
+
+  @Override
+  public BytesRef lookupOrd(int ord) {
+    return values.lookupOrd(ord);
+  }
+
+  @Override
+  public int getValueCount() {
+    return values.getValueCount();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValues.java
new file mode 100644
index 0000000..42e9a5d
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValues.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.index;
+
+
+/**
+ * A list of per-document numeric values, sorted 
+ * according to {@link Long#compare(long, long)}.
+ *
+ * @deprecated Use {@link SortedNumericDocValues} instead.
+ */
+@Deprecated
+public abstract class LegacySortedNumericDocValues {
+  
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected LegacySortedNumericDocValues() {}
+
+  /** 
+   * Positions to the specified document 
+   */
+  public abstract void setDocument(int doc);
+  
+  /** 
+   * Retrieve the value for the current document at the specified index. 
+   * An index ranges from {@code 0} to {@code count()-1}. 
+   */
+  public abstract long valueAt(int index);
+  
+  /** 
+   * Retrieves the count of values for the current document. 
+   * This may be zero if a document has no values.
+   */
+  public abstract int count();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
new file mode 100644
index 0000000..bf3c6cd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+/**
+ * Wraps a {@link LegacySortedNumericDocValues} into a {@link SortedNumericDocValues}.
+ *
+ * @deprecated Implement {@link SortedNumericDocValues} directly.
+ */
+@Deprecated
+public final class LegacySortedNumericDocValuesWrapper extends SortedNumericDocValues {
+  private final LegacySortedNumericDocValues values;
+  private final int maxDoc;
+  private int docID = -1;
+  private int upto;
+  
+  public LegacySortedNumericDocValuesWrapper(LegacySortedNumericDocValues values, int maxDoc) {
+    this.values = values;
+    this.maxDoc = maxDoc;
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() {
+    assert docID != NO_MORE_DOCS;
+    while (true) {
+      docID++;
+      if (docID == maxDoc) {
+        docID = NO_MORE_DOCS;
+        break;
+      }
+      values.setDocument(docID);
+      if (values.count() != 0) {
+        break;
+      }
+    }
+    upto = 0;
+    return docID;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target < docID) {
+      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
+    }
+    if (target >= maxDoc) {
+      docID = NO_MORE_DOCS;
+    } else {
+      docID = target-1;
+      nextDoc();
+    }
+    return docID;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  @Override
+  public long nextValue() {
+    return values.valueAt(upto++);
+  }
+
+  @Override
+  public int docValueCount() {
+    return values.count();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
new file mode 100644
index 0000000..dae1179
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.index;
+
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A per-document set of presorted byte[] values.
+ * <p>
+ * Per-Document values in a SortedDocValues are deduplicated, dereferenced,
+ * and sorted into a dictionary of unique values. A pointer to the
+ * dictionary value (ordinal) can be retrieved for each document. Ordinals
+ * are dense and in increasing sorted order.
+ *
+ * @deprecated Use {@link SortedSetDocValues} instead.
+ */
+@Deprecated
+public abstract class LegacySortedSetDocValues {
+  
+  /** Sole constructor. (For invocation by subclass 
+   * constructors, typically implicit.) */
+  protected LegacySortedSetDocValues() {}
+
+  /** When returned by {@link #nextOrd()} it means there are no more 
+   *  ordinals for the document.
+   */
+  public static final long NO_MORE_ORDS = -1;
+
+  /** 
+   * Returns the next ordinal for the current document (previously
+   * set by {@link #setDocument(int)}.
+   * @return next ordinal for the document, or {@link #NO_MORE_ORDS}. 
+   *         ordinals are dense, start at 0, then increment by 1 for 
+   *         the next value in sorted order. 
+   */
+  public abstract long nextOrd();
+  
+  /** 
+   * Sets iteration to the specified docID 
+   * @param docID document ID 
+   */
+  public abstract void setDocument(int docID);
+
+  /** Retrieves the value for the specified ordinal. The returned
+   * {@link BytesRef} may be re-used across calls to lookupOrd so make sure to
+   * {@link BytesRef#deepCopyOf(BytesRef) copy it} if you want to keep it
+   * around.
+   * @param ord ordinal to lookup
+   * @see #nextOrd
+   */
+  public abstract BytesRef lookupOrd(long ord);
+
+  /**
+   * Returns the number of unique values.
+   * @return number of unique values in this SortedDocValues. This is
+   *         also equivalent to one plus the maximum ordinal.
+   */
+  public abstract long getValueCount();
+
+  /** If {@code key} exists, returns its ordinal, else
+   *  returns {@code -insertionPoint-1}, like {@code
+   *  Arrays.binarySearch}.
+   *
+   *  @param key Key to look up
+   **/
+  public long lookupTerm(BytesRef key) {
+    long low = 0;
+    long high = getValueCount()-1;
+
+    while (low <= high) {
+      long mid = (low + high) >>> 1;
+      final BytesRef term = lookupOrd(mid);
+      int cmp = term.compareTo(key);
+
+      if (cmp < 0) {
+        low = mid + 1;
+      } else if (cmp > 0) {
+        high = mid - 1;
+      } else {
+        return mid; // key found
+      }
+    }
+
+    return -(low + 1);  // key not found.
+  }
+  
+  /** 
+   * Returns a {@link TermsEnum} over the values.
+   * The enum supports {@link TermsEnum#ord()} and {@link TermsEnum#seekExact(long)}.
+   */
+  public TermsEnum termsEnum() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
new file mode 100644
index 0000000..45d12d2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Wraps a {@link LegacySortedSetDocValues} into a {@link SortedSetDocValues}.
+ *
+ * @deprecated Implement {@link SortedSetDocValues} directly.
+ */
+@Deprecated
+public final class LegacySortedSetDocValuesWrapper extends SortedSetDocValues {
+  private final LegacySortedSetDocValues values;
+  private final int maxDoc;
+  private int docID = -1;
+  private long ord;
+  
+  public LegacySortedSetDocValuesWrapper(LegacySortedSetDocValues values, int maxDoc) {
+    this.values = values;
+    this.maxDoc = maxDoc;
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public int nextDoc() {
+    assert docID != NO_MORE_DOCS;
+    docID++;
+    while (docID < maxDoc) {
+      values.setDocument(docID);
+      ord = values.nextOrd();
+      if (ord != NO_MORE_ORDS) {
+        return docID;
+      }
+      docID++;
+    }
+    docID = NO_MORE_DOCS;
+    return NO_MORE_DOCS;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (target < docID) {
+      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
+    }
+    if (target >= maxDoc) {
+      this.docID = NO_MORE_DOCS;
+    } else {
+      this.docID = target-1;
+      nextDoc();
+    }
+    return docID;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  @Override
+  public long nextOrd() {
+    long result = ord;
+    if (result != NO_MORE_ORDS) {
+      ord = values.nextOrd();
+    }
+    return result;
+  }
+
+  @Override
+  public BytesRef lookupOrd(long ord) {
+    return values.lookupOrd((int) ord);
+  }
+
+  @Override
+  public long getValueCount() {
+    return values.getValueCount();
+  }
+
+  @Override
+  public String toString() {
+    return "LegacySortedSetDocValuesWrapper(" + values + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java
index bcecf2f..4c06b38 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java
@@ -161,21 +161,6 @@ class MergeReaderWrapper extends LeafReader {
   }
 
   @Override
-  public Bits getDocsWithField(String field) throws IOException {
-    ensureOpen();
-    FieldInfo fi = getFieldInfos().fieldInfo(field);
-    if (fi == null) {
-      // Field does not exist
-      return null;
-    }
-    if (fi.getDocValuesType() == DocValuesType.NONE) {
-      // Field was not indexed with doc values
-      return null;
-    }
-    return docValues.getDocsWithField(fi);
-  }
-
-  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     FieldInfo fi = getFieldInfos().fieldInfo(field);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java b/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
index af4dcfc..4054e90 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
@@ -27,7 +27,6 @@ import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.LongValues;
@@ -73,39 +72,86 @@ public class MultiDocValues {
       return null;
     }
 
-    boolean anyReal = false;
-    final NumericDocValues[] values = new NumericDocValues[size];
-    final int[] starts = new int[size+1];
-    for (int i = 0; i < size; i++) {
-      LeafReaderContext context = leaves.get(i);
-      NumericDocValues v = context.reader().getNormValues(field);
-      if (v == null) {
-        v = DocValues.emptyNumeric();
-      } else {
-        anyReal = true;
+    return new NumericDocValues() {
+      private int nextLeaf;
+      private NumericDocValues currentValues;
+      private LeafReaderContext currentLeaf;
+      private int docID = -1;
+
+      @Override
+      public int nextDoc() throws IOException {
+        while (true) {
+          if (currentValues == null) {
+            if (nextLeaf == leaves.size()) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            currentLeaf = leaves.get(nextLeaf);
+            currentValues = currentLeaf.reader().getNormValues(field);
+            nextLeaf++;
+            continue;
+          }
+
+          int newDocID = currentValues.nextDoc();
+
+          if (newDocID == NO_MORE_DOCS) {
+            currentValues = null;
+            continue;
+          } else {
+            docID = currentLeaf.docBase + newDocID;
+            return docID;
+          }
+        }
+      }
+        
+      @Override
+      public int docID() {
+        return docID;
       }
-      values[i] = v;
-      starts[i] = context.docBase;
-    }
-    starts[size] = r.maxDoc();
-    
-    assert anyReal;
 
-    return new NumericDocValues() {
       @Override
-      public long get(int docID) {
-        int subIndex = ReaderUtil.subIndex(docID, starts);
-        return values[subIndex].get(docID - starts[subIndex]);
+      public int advance(int targetDocID) throws IOException {
+        if (targetDocID <= docID) {
+          throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+        }
+        int readerIndex = ReaderUtil.subIndex(targetDocID, leaves);
+        if (readerIndex >= nextLeaf) {
+          if (readerIndex == leaves.size()) {
+            currentValues = null;
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentLeaf = leaves.get(readerIndex);
+          currentValues = currentLeaf.reader().getNormValues(field);
+          if (currentValues == null) {
+            return nextDoc();
+          }
+          nextLeaf = readerIndex+1;
+        }
+        int newDocID = currentValues.advance(targetDocID - currentLeaf.docBase);
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          return nextDoc();
+        } else {
+          docID = currentLeaf.docBase + newDocID;
+          return docID;
+        }
+      }
+
+      @Override
+      public long longValue() {
+        return currentValues.longValue();
+      }
+
+      @Override
+      public long cost() {
+        // TODO
+        return 0;
       }
     };
   }
 
-  /** Returns a NumericDocValues for a reader's docvalues (potentially merging on-the-fly) 
-   * <p>
-   * This is a slow way to access numeric values. Instead, access them per-segment
-   * with {@link LeafReader#getNumericDocValues(String)}
-   * </p> 
-   * */
+  /** Returns a NumericDocValues for a reader's docvalues (potentially merging on-the-fly) */
   public static NumericDocValues getNumericValues(final IndexReader r, final String field) throws IOException {
     final List<LeafReaderContext> leaves = r.leaves();
     final int size = leaves.size();
@@ -116,124 +162,204 @@ public class MultiDocValues {
     }
 
     boolean anyReal = false;
-    final NumericDocValues[] values = new NumericDocValues[size];
-    final int[] starts = new int[size+1];
-    for (int i = 0; i < size; i++) {
-      LeafReaderContext context = leaves.get(i);
-      NumericDocValues v = context.reader().getNumericDocValues(field);
-      if (v == null) {
-        v = DocValues.emptyNumeric();
-      } else {
-        anyReal = true;
+    for(LeafReaderContext leaf : leaves) {
+      FieldInfo fieldInfo = leaf.reader().getFieldInfos().fieldInfo(field);
+      if (fieldInfo != null) {
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        if (dvType == DocValuesType.NUMERIC) {
+          anyReal = true;
+          break;
+        }
       }
-      values[i] = v;
-      starts[i] = context.docBase;
     }
-    starts[size] = r.maxDoc();
 
-    if (!anyReal) {
+    if (anyReal == false) {
       return null;
-    } else {
-      return new NumericDocValues() {
-        @Override
-        public long get(int docID) {
-          int subIndex = ReaderUtil.subIndex(docID, starts);
-          return values[subIndex].get(docID - starts[subIndex]);
-        }
-      };
-    }
-  }
-  
-  /** Returns a Bits for a reader's docsWithField (potentially merging on-the-fly) 
-   * <p>
-   * This is a slow way to access this bitset. Instead, access them per-segment
-   * with {@link LeafReader#getDocsWithField(String)}
-   * </p> 
-   * */
-  public static Bits getDocsWithField(final IndexReader r, final String field) throws IOException {
-    final List<LeafReaderContext> leaves = r.leaves();
-    final int size = leaves.size();
-    if (size == 0) {
-      return null;
-    } else if (size == 1) {
-      return leaves.get(0).reader().getDocsWithField(field);
     }
+    
 
-    boolean anyReal = false;
-    boolean anyMissing = false;
-    final Bits[] values = new Bits[size];
-    final int[] starts = new int[size+1];
-    for (int i = 0; i < size; i++) {
-      LeafReaderContext context = leaves.get(i);
-      Bits v = context.reader().getDocsWithField(field);
-      if (v == null) {
-        v = new Bits.MatchNoBits(context.reader().maxDoc());
-        anyMissing = true;
-      } else {
-        anyReal = true;
-        if (v instanceof Bits.MatchAllBits == false) {
-          anyMissing = true;
+    return new NumericDocValues() {
+      private int nextLeaf;
+      private NumericDocValues currentValues;
+      private LeafReaderContext currentLeaf;
+      private int docID = -1;
+
+      @Override
+      public int docID() {
+        return docID;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        while (true) {
+          while (currentValues == null) {
+            if (nextLeaf == leaves.size()) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            currentLeaf = leaves.get(nextLeaf);
+            currentValues = currentLeaf.reader().getNumericDocValues(field);
+            nextLeaf++;
+          }
+
+          int newDocID = currentValues.nextDoc();
+
+          if (newDocID == NO_MORE_DOCS) {
+            currentValues = null;
+            continue;
+          } else {
+            docID = currentLeaf.docBase + newDocID;
+            return docID;
+          }
+        }
+      }
+        
+      @Override
+      public int advance(int targetDocID) throws IOException {
+        if (targetDocID <= docID) {
+          throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+        }
+        int readerIndex = ReaderUtil.subIndex(targetDocID, leaves);
+        if (readerIndex >= nextLeaf) {
+          if (readerIndex == leaves.size()) {
+            currentValues = null;
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentLeaf = leaves.get(readerIndex);
+          currentValues = currentLeaf.reader().getNumericDocValues(field);
+          nextLeaf = readerIndex+1;
+          if (currentValues == null) {
+            return nextDoc();
+          }
+        }
+        int newDocID = currentValues.advance(targetDocID - currentLeaf.docBase);
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          return nextDoc();
+        } else {
+          docID = currentLeaf.docBase + newDocID;
+          return docID;
         }
       }
-      values[i] = v;
-      starts[i] = context.docBase;
-    }
-    starts[size] = r.maxDoc();
 
-    if (!anyReal) {
-      return null;
-    } else if (!anyMissing) {
-      return new Bits.MatchAllBits(r.maxDoc());
-    } else {
-      return new MultiBits(values, starts, false);
-    }
+      @Override
+      public long longValue() {
+        return currentValues.longValue();
+      }
+
+      @Override
+      public long cost() {
+        // TODO
+        return 0;
+      }
+    };
   }
 
-  /** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly)
-   * <p>
-   * This is a slow way to access binary values. Instead, access them per-segment
-   * with {@link LeafReader#getBinaryDocValues(String)}
-   * </p>  
-   */
+  /** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly) */
   public static BinaryDocValues getBinaryValues(final IndexReader r, final String field) throws IOException {
     final List<LeafReaderContext> leaves = r.leaves();
     final int size = leaves.size();
-    
     if (size == 0) {
       return null;
     } else if (size == 1) {
       return leaves.get(0).reader().getBinaryDocValues(field);
     }
-    
+
     boolean anyReal = false;
-    final BinaryDocValues[] values = new BinaryDocValues[size];
-    final int[] starts = new int[size+1];
-    for (int i = 0; i < size; i++) {
-      LeafReaderContext context = leaves.get(i);
-      BinaryDocValues v = context.reader().getBinaryDocValues(field);
-      if (v == null) {
-        v = DocValues.emptyBinary();
-      } else {
-        anyReal = true;
+    for(LeafReaderContext leaf : leaves) {
+      FieldInfo fieldInfo = leaf.reader().getFieldInfos().fieldInfo(field);
+      if (fieldInfo != null) {
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        if (dvType == DocValuesType.BINARY) {
+          anyReal = true;
+          break;
+        }
       }
-      values[i] = v;
-      starts[i] = context.docBase;
     }
-    starts[size] = r.maxDoc();
-    
-    if (!anyReal) {
+
+    if (anyReal == false) {
       return null;
-    } else {
-      return new BinaryDocValues() {
-        @Override
-        public BytesRef get(int docID) {
-          int subIndex = ReaderUtil.subIndex(docID, starts);
-          return values[subIndex].get(docID - starts[subIndex]);
-        }
-      };
     }
+
+    return new BinaryDocValues() {
+      private int nextLeaf;
+      private BinaryDocValues currentValues;
+      private LeafReaderContext currentLeaf;
+      private int docID = -1;
+
+      @Override
+      public int nextDoc() throws IOException {
+        while (true) {
+          while (currentValues == null) {
+            if (nextLeaf == leaves.size()) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            currentLeaf = leaves.get(nextLeaf);
+            currentValues = currentLeaf.reader().getBinaryDocValues(field);
+            nextLeaf++;
+          }
+
+          int newDocID = currentValues.nextDoc();
+
+          if (newDocID == NO_MORE_DOCS) {
+            currentValues = null;
+            continue;
+          } else {
+            docID = currentLeaf.docBase + newDocID;
+            return docID;
+          }
+        }
+      }
+        
+      @Override
+      public int docID() {
+        return docID;
+      }
+
+      @Override
+      public int advance(int targetDocID) throws IOException {
+        if (targetDocID <= docID) {
+          throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+        }
+        int readerIndex = ReaderUtil.subIndex(targetDocID, leaves);
+        if (readerIndex >= nextLeaf) {
+          if (readerIndex == leaves.size()) {
+            currentValues = null;
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentLeaf = leaves.get(readerIndex);
+          currentValues = currentLeaf.reader().getBinaryDocValues(field);
+          nextLeaf = readerIndex+1;
+          if (currentValues == null) {
+            return nextDoc();
+          }
+        }
+        int newDocID = currentValues.advance(targetDocID - currentLeaf.docBase);
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          return nextDoc();
+        } else {
+          docID = currentLeaf.docBase + newDocID;
+          return docID;
+        }
+      }
+
+      @Override
+      public BytesRef binaryValue() {
+        return currentValues.binaryValue();
+      }
+
+      @Override
+      public long cost() {
+        // TODO
+        return 0;
+      }
+    };
   }
-  
+
   /** Returns a SortedNumericDocValues for a reader's docvalues (potentially merging on-the-fly) 
    * <p>
    * This is a slow way to access sorted numeric values. Instead, access them per-segment
@@ -252,6 +378,7 @@ public class MultiDocValues {
     boolean anyReal = false;
     final SortedNumericDocValues[] values = new SortedNumericDocValues[size];
     final int[] starts = new int[size+1];
+    long totalCost = 0;
     for (int i = 0; i < size; i++) {
       LeafReaderContext context = leaves.get(i);
       SortedNumericDocValues v = context.reader().getSortedNumericDocValues(field);
@@ -262,33 +389,93 @@ public class MultiDocValues {
       }
       values[i] = v;
       starts[i] = context.docBase;
+      totalCost += v.cost();
     }
     starts[size] = r.maxDoc();
 
-    if (!anyReal) {
+    if (anyReal == false) {
       return null;
-    } else {
-      return new SortedNumericDocValues() {
-        SortedNumericDocValues current;
-
-        @Override
-        public void setDocument(int doc) {
-          int subIndex = ReaderUtil.subIndex(doc, starts);
-          current = values[subIndex];
-          current.setDocument(doc - starts[subIndex]);
-        }
+    }
 
-        @Override
-        public long valueAt(int index) {
-          return current.valueAt(index);
-        }
+    final long finalTotalCost = totalCost;
+    
+    return new SortedNumericDocValues() {
+      private int nextLeaf;
+      private SortedNumericDocValues currentValues;
+      private LeafReaderContext currentLeaf;
+      private int docID = -1;
 
-        @Override
-        public int count() {
-          return current.count();
+      @Override
+      public int nextDoc() throws IOException {
+        while (true) {
+          if (currentValues == null) {
+            if (nextLeaf == leaves.size()) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            currentLeaf = leaves.get(nextLeaf);
+            currentValues = values[nextLeaf];
+            nextLeaf++;
+          }
+
+          int newDocID = currentValues.nextDoc();
+
+          if (newDocID == NO_MORE_DOCS) {
+            currentValues = null;
+            continue;
+          } else {
+            docID = currentLeaf.docBase + newDocID;
+            return docID;
+          }
         }
-      };
-    }
+      }
+        
+      @Override
+      public int docID() {
+        return docID;
+      }
+        
+      @Override
+      public int advance(int targetDocID) throws IOException {
+        if (targetDocID <= docID) {
+          throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+        }
+        int readerIndex = ReaderUtil.subIndex(targetDocID, leaves);
+        if (readerIndex >= nextLeaf) {
+          if (readerIndex == leaves.size()) {
+            currentValues = null;
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentLeaf = leaves.get(readerIndex);
+          currentValues = values[readerIndex];
+          nextLeaf = readerIndex+1;
+        }
+        int newDocID = currentValues.advance(targetDocID - currentLeaf.docBase);
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          return nextDoc();
+        } else {
+          docID = currentLeaf.docBase + newDocID;
+          return docID;
+        }
+      }
+
+      @Override
+      public long cost() {
+        return finalTotalCost;
+      }
+      
+      @Override
+      public int docValueCount() {
+        return currentValues.docValueCount();
+      }
+
+      @Override
+      public long nextValue() throws IOException {
+        return currentValues.nextValue();
+      }
+    };
   }
   
   /** Returns a SortedDocValues for a reader's docvalues (potentially doing extremely slow things).
@@ -310,6 +497,7 @@ public class MultiDocValues {
     boolean anyReal = false;
     final SortedDocValues[] values = new SortedDocValues[size];
     final int[] starts = new int[size+1];
+    long totalCost = 0;
     for (int i = 0; i < size; i++) {
       LeafReaderContext context = leaves.get(i);
       SortedDocValues v = context.reader().getSortedDocValues(field);
@@ -317,17 +505,18 @@ public class MultiDocValues {
         v = DocValues.emptySorted();
       } else {
         anyReal = true;
+        totalCost += v.cost();
       }
       values[i] = v;
       starts[i] = context.docBase;
     }
     starts[size] = r.maxDoc();
     
-    if (!anyReal) {
+    if (anyReal == false) {
       return null;
     } else {
       OrdinalMap mapping = OrdinalMap.build(r.getCoreCacheKey(), values, PackedInts.DEFAULT);
-      return new MultiSortedDocValues(values, starts, mapping);
+      return new MultiSortedDocValues(values, starts, mapping, totalCost);
     }
   }
   
@@ -350,6 +539,7 @@ public class MultiDocValues {
     boolean anyReal = false;
     final SortedSetDocValues[] values = new SortedSetDocValues[size];
     final int[] starts = new int[size+1];
+    long totalCost = 0;
     for (int i = 0; i < size; i++) {
       LeafReaderContext context = leaves.get(i);
       SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
@@ -357,17 +547,18 @@ public class MultiDocValues {
         v = DocValues.emptySortedSet();
       } else {
         anyReal = true;
+        totalCost += v.cost();
       }
       values[i] = v;
       starts[i] = context.docBase;
     }
     starts[size] = r.maxDoc();
     
-    if (!anyReal) {
+    if (anyReal == false) {
       return null;
     } else {
       OrdinalMap mapping = OrdinalMap.build(r.getCoreCacheKey(), values, PackedInts.DEFAULT);
-      return new MultiSortedSetDocValues(values, starts, mapping);
+      return new MultiSortedSetDocValues(values, starts, mapping, totalCost);
     }
   }
 
@@ -659,20 +850,81 @@ public class MultiDocValues {
     public final SortedDocValues values[];
     /** ordinal map mapping ords from <code>values</code> to global ord space */
     public final OrdinalMap mapping;
+    private final long totalCost;
+
+    private int nextLeaf;
+    private SortedDocValues currentValues;
+    private int currentDocStart;
+    private int docID = -1;    
   
     /** Creates a new MultiSortedDocValues over <code>values</code> */
-    public MultiSortedDocValues(SortedDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
+    public MultiSortedDocValues(SortedDocValues values[], int docStarts[], OrdinalMap mapping, long totalCost) throws IOException {
       assert docStarts.length == values.length + 1;
       this.values = values;
       this.docStarts = docStarts;
       this.mapping = mapping;
+      this.totalCost = totalCost;
     }
        
     @Override
-    public int getOrd(int docID) {
-      int subIndex = ReaderUtil.subIndex(docID, docStarts);
-      int segmentOrd = values[subIndex].getOrd(docID - docStarts[subIndex]);
-      return segmentOrd == -1 ? segmentOrd : (int) mapping.getGlobalOrds(subIndex).get(segmentOrd);
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while (true) {
+        while (currentValues == null) {
+          if (nextLeaf == values.length) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentDocStart = docStarts[nextLeaf];
+          currentValues = values[nextLeaf];
+          nextLeaf++;
+        }
+
+        int newDocID = currentValues.nextDoc();
+
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          continue;
+        } else {
+          docID = currentDocStart + newDocID;
+          return docID;
+        }
+      }
+    }
+
+    @Override
+    public int advance(int targetDocID) throws IOException {
+      if (targetDocID <= docID) {
+        throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+      }
+      int readerIndex = ReaderUtil.subIndex(targetDocID, docStarts);
+      if (readerIndex >= nextLeaf) {
+        if (readerIndex == values.length) {
+          currentValues = null;
+          docID = NO_MORE_DOCS;
+          return docID;
+        }
+        currentDocStart = docStarts[readerIndex];
+        currentValues = values[readerIndex];
+        nextLeaf = readerIndex+1;
+      }
+      int newDocID = currentValues.advance(targetDocID - currentDocStart);
+      if (newDocID == NO_MORE_DOCS) {
+        currentValues = null;
+        return nextDoc();
+      } else {
+        docID = currentDocStart + newDocID;
+        return docID;
+      }
+    }
+    
+    @Override
+    public int ordValue() {
+      return (int) mapping.getGlobalOrds(nextLeaf-1).get(currentValues.ordValue());
     }
  
     @Override
@@ -686,6 +938,11 @@ public class MultiDocValues {
     public int getValueCount() {
       return (int) mapping.getValueCount();
     }
+
+    @Override
+    public long cost() {
+      return totalCost;
+    }
   }
   
   /** 
@@ -699,34 +956,88 @@ public class MultiDocValues {
     public final SortedSetDocValues values[];
     /** ordinal map mapping ords from <code>values</code> to global ord space */
     public final OrdinalMap mapping;
-    int currentSubIndex;
-    LongValues currentGlobalOrds;
-    
+    private final long totalCost;
+
+    private int nextLeaf;
+    private SortedSetDocValues currentValues;
+    private int currentDocStart;
+    private int docID = -1;    
+
     /** Creates a new MultiSortedSetDocValues over <code>values</code> */
-    public MultiSortedSetDocValues(SortedSetDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
+    public MultiSortedSetDocValues(SortedSetDocValues values[], int docStarts[], OrdinalMap mapping, long totalCost) throws IOException {
       assert docStarts.length == values.length + 1;
       this.values = values;
       this.docStarts = docStarts;
       this.mapping = mapping;
+      this.totalCost = totalCost;
     }
     
     @Override
-    public long nextOrd() {
-      long segmentOrd = values[currentSubIndex].nextOrd();
-      if (segmentOrd == NO_MORE_ORDS) {
-        return segmentOrd;
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while (true) {
+        while (currentValues == null) {
+          if (nextLeaf == values.length) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          }
+          currentDocStart = docStarts[nextLeaf];
+          currentValues = values[nextLeaf];
+          nextLeaf++;
+        }
+
+        int newDocID = currentValues.nextDoc();
+
+        if (newDocID == NO_MORE_DOCS) {
+          currentValues = null;
+          continue;
+        } else {
+          docID = currentDocStart + newDocID;
+          return docID;
+        }
+      }
+    }
+
+    @Override
+    public int advance(int targetDocID) throws IOException {
+      if (targetDocID <= docID) {
+        throw new IllegalArgumentException("can only advance beyond current document: on docID=" + docID + " but targetDocID=" + targetDocID);
+      }
+      int readerIndex = ReaderUtil.subIndex(targetDocID, docStarts);
+      if (readerIndex >= nextLeaf) {
+        if (readerIndex == values.length) {
+          currentValues = null;
+          docID = NO_MORE_DOCS;
+          return docID;
+        }
+        currentDocStart = docStarts[readerIndex];
+        currentValues = values[readerIndex];
+        nextLeaf = readerIndex+1;
+      }
+      int newDocID = currentValues.advance(targetDocID - currentDocStart);
+      if (newDocID == NO_MORE_DOCS) {
+        currentValues = null;
+        return nextDoc();
       } else {
-        return currentGlobalOrds.get(segmentOrd);
+        docID = currentDocStart + newDocID;
+        return docID;
       }
     }
 
     @Override
-    public void setDocument(int docID) {
-      currentSubIndex = ReaderUtil.subIndex(docID, docStarts);
-      currentGlobalOrds = mapping.getGlobalOrds(currentSubIndex);
-      values[currentSubIndex].setDocument(docID - docStarts[currentSubIndex]);
+    public long nextOrd() throws IOException {
+      long segmentOrd = currentValues.nextOrd();
+      if (segmentOrd == NO_MORE_ORDS) {
+        return segmentOrd;
+      } else {
+        return mapping.getGlobalOrds(nextLeaf-1).get(segmentOrd);
+      }
     }
- 
+
     @Override
     public BytesRef lookupOrd(long ord) {
       int subIndex = mapping.getFirstSegmentNumber(ord);
@@ -738,5 +1049,10 @@ public class MultiDocValues {
     public long getValueCount() {
       return mapping.getValueCount();
     }
+
+    @Override
+    public long cost() {
+      return totalCost;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
index 51cabab..ee969c7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
@@ -18,18 +18,18 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.index.MergeState.DocMap;
-import org.apache.lucene.index.MergeState;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
+@SuppressWarnings({"unchecked","rawtypes"})
 final class MultiSorter {
   
   /** Does a merge sort of the leaves of the incoming reader, returning {@link DocMap} to map each leaf's
@@ -39,9 +39,9 @@ final class MultiSorter {
     // TODO: optimize if only 1 reader is incoming, though that's a rare case
 
     SortField fields[] = sort.getSort();
-    final CrossReaderComparator[] comparators = new CrossReaderComparator[fields.length];
+    final ComparableProvider[][] comparables = new ComparableProvider[fields.length][];
     for(int i=0;i<fields.length;i++) {
-      comparators[i] = getComparator(readers, fields[i]);
+      comparables[i] = getComparableProviders(readers, fields[i]);
     }
 
     int leafCount = readers.size();
@@ -49,8 +49,8 @@ final class MultiSorter {
     PriorityQueue<LeafAndDocID> queue = new PriorityQueue<LeafAndDocID>(leafCount) {
         @Override
         public boolean lessThan(LeafAndDocID a, LeafAndDocID b) {
-          for(int i=0;i<comparators.length;i++) {
-            int cmp = comparators[i].compare(a.readerIndex, a.docID, b.readerIndex, b.docID);
+          for(int i=0;i<comparables.length;i++) {
+            int cmp = a.values[i].compareTo(b.values[i]);
             if (cmp != 0) {
               return cmp < 0;
             }
@@ -59,8 +59,9 @@ final class MultiSorter {
           // tie-break by docID natural order:
           if (a.readerIndex != b.readerIndex) {
             return a.readerIndex < b.readerIndex;
+          } else {
+            return a.docID < b.docID;
           }
-          return a.docID < b.docID;
         }
     };
 
@@ -68,10 +69,16 @@ final class MultiSorter {
 
     for(int i=0;i<leafCount;i++) {
       CodecReader reader = readers.get(i);
-      queue.add(new LeafAndDocID(i, reader.getLiveDocs(), reader.maxDoc()));
+      LeafAndDocID leaf = new LeafAndDocID(i, reader.getLiveDocs(), reader.maxDoc(), comparables.length);
+      for(int j=0;j<comparables.length;j++) {
+        leaf.values[j] = comparables[j][i].getComparable(leaf.docID);
+        assert leaf.values[j] != null;
+      }
+      queue.add(leaf);
       builders[i] = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
     }
 
+    // merge sort:
     int mappedDocID = 0;
     while (queue.size() != 0) {
       LeafAndDocID top = queue.top();
@@ -81,6 +88,10 @@ final class MultiSorter {
       }
       top.docID++;
       if (top.docID < top.maxDoc) {
+        for(int j=0;j<comparables.length;j++) {
+          top.values[j] = comparables[j][top.readerIndex].getComparable(top.docID);
+          assert top.values[j] != null;
+        }
         queue.updateTop();
       } else {
         queue.pop();
@@ -110,32 +121,41 @@ final class MultiSorter {
     final int readerIndex;
     final Bits liveDocs;
     final int maxDoc;
+    final Comparable[] values;
     int docID;
 
-    public LeafAndDocID(int readerIndex, Bits liveDocs, int maxDoc) {
+    public LeafAndDocID(int readerIndex, Bits liveDocs, int maxDoc, int numComparables) {
       this.readerIndex = readerIndex;
       this.liveDocs = liveDocs;
       this.maxDoc = maxDoc;
+      this.values = new Comparable[numComparables];
     }
   }
 
-  private interface CrossReaderComparator {
-    public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB);
+  /** Returns an object for this docID whose .compareTo represents the requested {@link SortField} sort order. */
+  private interface ComparableProvider {
+    public Comparable getComparable(int docID) throws IOException;
   }
 
-  private static CrossReaderComparator getComparator(List<CodecReader> readers, SortField sortField) throws IOException {
+  /** Returns {@code ComparableProvider}s for the provided readers to represent the requested {@link SortField} sort order. */
+  private static ComparableProvider[] getComparableProviders(List<CodecReader> readers, SortField sortField) throws IOException {
+
+    ComparableProvider[] providers = new ComparableProvider[readers.size()];
+
     switch(sortField.getType()) {
 
     case STRING:
       {
         // this uses the efficient segment-local ordinal map:
-        MultiReader multiReader = new MultiReader(readers.toArray(new LeafReader[readers.size()]));
-        final SortedDocValues sorted = MultiDocValues.getSortedValues(multiReader, sortField.getField());
-        final int[] docStarts = new int[readers.size()];
-        List<LeafReaderContext> leaves = multiReader.leaves();
+        final SortedDocValues[] values = new SortedDocValues[readers.size()];
         for(int i=0;i<readers.size();i++) {
-          docStarts[i] = leaves.get(i).docBase;
+          SortedDocValues v = readers.get(i).getSortedDocValues(sortField.getField());
+          if (v == null) {
+            v = DocValues.emptySorted();
+          }
+          values[i] = v;
         }
+        MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(null, values, PackedInts.DEFAULT);
         final int missingOrd;
         if (sortField.getMissingValue() == SortField.STRING_LAST) {
           missingOrd = Integer.MAX_VALUE;
@@ -150,31 +170,41 @@ final class MultiSorter {
           reverseMul = 1;
         }
 
-        return new CrossReaderComparator() {
-          @Override
-          public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
-            int ordA = sorted.getOrd(docStarts[readerIndexA] + docIDA);
-            if (ordA == -1) {
-              ordA = missingOrd;
-            }
-            int ordB = sorted.getOrd(docStarts[readerIndexB] + docIDB);
-            if (ordB == -1) {
-              ordB = missingOrd;
-            }
-            return reverseMul * Integer.compare(ordA, ordB);
-          }
-        };
+        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+          final SortedDocValues readerValues = values[readerIndex];
+          final LongValues globalOrds = ordinalMap.getGlobalOrds(readerIndex);
+          providers[readerIndex] = new ComparableProvider() {
+              // used only by assert:
+              int lastDocID = -1;
+              private boolean docsInOrder(int docID) {
+                if (docID < lastDocID) {
+                  throw new AssertionError("docs must be sent in order, but lastDocID=" + lastDocID + " vs docID=" + docID);
+                }
+                lastDocID = docID;
+                return true;
+              }
+              
+              @Override
+              public Comparable getComparable(int docID) throws IOException {
+                assert docsInOrder(docID);
+                int readerDocID = readerValues.docID();
+                if (readerDocID < docID) {
+                  readerDocID = readerValues.advance(docID);
+                }
+                if (readerDocID == docID) {
+                  // translate segment's ord to global ord space:
+                  return reverseMul * (int) globalOrds.get(readerValues.ordValue());
+                } else {
+                  return missingOrd;
+                }
+              }
+            };
+        }
       }
+      break;
 
     case LONG:
       {
-        List<NumericDocValues> values = new ArrayList<>();
-        List<Bits> docsWithFields = new ArrayList<>();
-        for(CodecReader reader : readers) {
-          values.add(DocValues.getNumeric(reader, sortField.getField()));
-          docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
-        }
-
         final int reverseMul;
         if (sortField.getReverse()) {
           reverseMul = -1;
@@ -182,44 +212,47 @@ final class MultiSorter {
           reverseMul = 1;
         }
 
-        final long missingValue;
-
+        final Long missingValue;
         if (sortField.getMissingValue() != null) {
           missingValue = (Long) sortField.getMissingValue();
         } else {
-          missingValue = 0;
+          missingValue = 0L;
         }
 
-        return new CrossReaderComparator() {
-          @Override
-          public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
-            long valueA;
-            if (docsWithFields.get(readerIndexA).get(docIDA)) {
-              valueA = values.get(readerIndexA).get(docIDA);
-            } else {
-              valueA = missingValue;
-            }
-
-            long valueB;
-            if (docsWithFields.get(readerIndexB).get(docIDB)) {
-              valueB = values.get(readerIndexB).get(docIDB);
-            } else {
-              valueB = missingValue;
-            }
-            return reverseMul * Long.compare(valueA, valueB);
-          }
-        };
+        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+          final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
+          
+          providers[readerIndex] = new ComparableProvider() {
+              // used only by assert:
+              int lastDocID = -1;
+              private boolean docsInOrder(int docID) {
+                if (docID < lastDocID) {
+                  throw new AssertionError("docs must be sent in order, but lastDocID=" + lastDocID + " vs docID=" + docID);
+                }
+                lastDocID = docID;
+                return true;
+              }
+              
+              @Override
+              public Comparable getComparable(int docID) throws IOException {
+                assert docsInOrder(docID);
+                int readerDocID = values.docID();
+                if (readerDocID < docID) {
+                  readerDocID = values.advance(docID);
+                }
+                if (readerDocID == docID) {
+                  return reverseMul * values.longValue();
+                } else {
+                  return missingValue;
+                }
+              }
+            };
+        }
       }
+      break;
 
     case INT:
       {
-        List<NumericDocValues> values = new ArrayList<>();
-        List<Bits> docsWithFields = new ArrayList<>();
-        for(CodecReader reader : readers) {
-          values.add(DocValues.getNumeric(reader, sortField.getField()));
-          docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
-        }
-
         final int reverseMul;
         if (sortField.getReverse()) {
           reverseMul = -1;
@@ -227,44 +260,47 @@ final class MultiSorter {
           reverseMul = 1;
         }
 
-        final int missingValue;
-
+        final Integer missingValue;
         if (sortField.getMissingValue() != null) {
           missingValue = (Integer) sortField.getMissingValue();
         } else {
           missingValue = 0;
         }
 
-        return new CrossReaderComparator() {
-          @Override
-          public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
-            int valueA;
-            if (docsWithFields.get(readerIndexA).get(docIDA)) {
-              valueA = (int) values.get(readerIndexA).get(docIDA);
-            } else {
-              valueA = missingValue;
-            }
-
-            int valueB;
-            if (docsWithFields.get(readerIndexB).get(docIDB)) {
-              valueB = (int) values.get(readerIndexB).get(docIDB);
-            } else {
-              valueB = missingValue;
-            }
-            return reverseMul * Integer.compare(valueA, valueB);
-          }
-        };
+        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+          final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
+
+          providers[readerIndex] = new ComparableProvider() {
+              // used only by assert:
+              int lastDocID = -1;
+              private boolean docsInOrder(int docID) {
+                if (docID < lastDocID) {
+                  throw new AssertionError("docs must be sent in order, but lastDocID=" + lastDocID + " vs docID=" + docID);
+                }
+                lastDocID = docID;
+                return true;
+              }
+              
+              @Override
+              public Comparable getComparable(int docID) throws IOException {
+                assert docsInOrder(docID);
+                int readerDocID = values.docID();
+                if (readerDocID < docID) {
+                  readerDocID = values.advance(docID);
+                }
+                if (readerDocID == docID) {
+                  return reverseMul * (int) values.longValue();
+                } else {
+                  return missingValue;
+                }
+              }
+            };
+        }
       }
+      break;
 
     case DOUBLE:
       {
-        List<NumericDocValues> values = new ArrayList<>();
-        List<Bits> docsWithFields = new ArrayList<>();
-        for(CodecReader reader : readers) {
-          values.add(DocValues.getNumeric(reader, sortField.getField()));
-          docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
-        }
-
         final int reverseMul;
         if (sortField.getReverse()) {
           reverseMul = -1;
@@ -272,44 +308,47 @@ final class MultiSorter {
           reverseMul = 1;
         }
 
-        final double missingValue;
-
+        final Double missingValue;
         if (sortField.getMissingValue() != null) {
           missingValue = (Double) sortField.getMissingValue();
         } else {
           missingValue = 0.0;
         }
 
-        return new CrossReaderComparator() {
-          @Override
-          public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
-            double valueA;
-            if (docsWithFields.get(readerIndexA).get(docIDA)) {
-              valueA = Double.longBitsToDouble(values.get(readerIndexA).get(docIDA));
-            } else {
-              valueA = missingValue;
-            }
-
-            double valueB;
-            if (docsWithFields.get(readerIndexB).get(docIDB)) {
-              valueB = Double.longBitsToDouble(values.get(readerIndexB).get(docIDB));
-            } else {
-              valueB = missingValue;
-            }
-            return reverseMul * Double.compare(valueA, valueB);
-          }
-        };
+        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+          final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
+
+          providers[readerIndex] = new ComparableProvider() {
+              // used only by assert:
+              int lastDocID = -1;
+              private boolean docsInOrder(int docID) {
+                if (docID < lastDocID) {
+                  throw new AssertionError("docs must be sent in order, but lastDocID=" + lastDocID + " vs docID=" + docID);
+                }
+                lastDocID = docID;
+                return true;
+              }
+              
+              @Override
+              public Comparable getComparable(int docID) throws IOException {
+                assert docsInOrder(docID);
+                int readerDocID = values.docID();
+                if (readerDocID < docID) {
+                  readerDocID = values.advance(docID);
+                }
+                if (readerDocID == docID) {
+                  return reverseMul * Double.longBitsToDouble(values.longValue());
+                } else {
+                  return missingValue;
+                }
+              }
+            };
+        }
       }
+      break;
 
     case FLOAT:
       {
-        List<NumericDocValues> values = new ArrayList<>();
-        List<Bits> docsWithFields = new ArrayList<>();
-        for(CodecReader reader : readers) {
-          values.add(DocValues.getNumeric(reader, sortField.getField()));
-          docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
-        }
-
         final int reverseMul;
         if (sortField.getReverse()) {
           reverseMul = -1;
@@ -317,37 +356,49 @@ final class MultiSorter {
           reverseMul = 1;
         }
 
-        final float missingValue;
-
+        final Float missingValue;
         if (sortField.getMissingValue() != null) {
           missingValue = (Float) sortField.getMissingValue();
         } else {
           missingValue = 0.0f;
         }
 
-        return new CrossReaderComparator() {
-          @Override
-          public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
-            float valueA;
-            if (docsWithFields.get(readerIndexA).get(docIDA)) {
-              valueA = Float.intBitsToFloat((int) values.get(readerIndexA).get(docIDA));
-            } else {
-              valueA = missingValue;
-            }
-
-            float valueB;
-            if (docsWithFields.get(readerIndexB).get(docIDB)) {
-              valueB = Float.intBitsToFloat((int) values.get(readerIndexB).get(docIDB));
-            } else {
-              valueB = missingValue;
-            }
-            return reverseMul * Float.compare(valueA, valueB);
-          }
-        };
+        for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
+          final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
+
+          providers[readerIndex] = new ComparableProvider() {
+              // used only by assert:
+              int lastDocID = -1;
+              private boolean docsInOrder(int docID) {
+                if (docID < lastDocID) {
+                  throw new AssertionError("docs must be sent in order, but lastDocID=" + lastDocID + " vs docID=" + docID);
+                }
+                lastDocID = docID;
+                return true;
+              }
+              
+              @Override
+              public Comparable getComparable(int docID) throws IOException {
+                assert docsInOrder(docID);
+                int readerDocID = values.docID();
+                if (readerDocID < docID) {
+                  readerDocID = values.advance(docID);
+                }
+                if (readerDocID == docID) {
+                  return reverseMul * Float.intBitsToFloat((int) values.longValue());
+                } else {
+                  return missingValue;
+                }
+              }
+            };
+        }
       }
+      break;
 
     default:
       throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
     }
+
+    return providers;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
index a9797a1..c444661 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NormValuesWriter.java
@@ -18,10 +18,9 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
@@ -70,51 +69,79 @@ class NormValuesWriter {
     final PackedLongValues values = pending.build();
 
     normsConsumer.addNormsField(fieldInfo,
-                               new Iterable<Number>() {
-                                 @Override
-                                 public Iterator<Number> iterator() {
-                                   return new NumericIterator(maxDoc, values);
-                                 }
+                                new NormsProducer() {
+                                  @Override
+                                  public NumericDocValues getNorms(FieldInfo fieldInfo2) {
+                                   if (fieldInfo != NormValuesWriter.this.fieldInfo) {
+                                     throw new IllegalArgumentException("wrong fieldInfo");
+                                   }
+                                   return new BufferedNorms(maxDoc, values);
+                                  }
+
+                                  @Override
+                                  public void checkIntegrity() {
+                                  }
+
+                                  @Override
+                                  public void close() {
+                                  }
+                                  
+                                  @Override
+                                  public long ramBytesUsed() {
+                                    return 0;
+                                  }
                                });
   }
 
+  // TODO: norms should only visit docs that had a field indexed!!
+  
   // iterates over the values we have in ram
-  private static class NumericIterator implements Iterator<Number> {
+  private static class BufferedNorms extends NumericDocValues {
     final PackedLongValues.Iterator iter;
     final int size;
     final int maxDoc;
-    int upto;
+    private int docID = -1;
+    private long value;
     
-    NumericIterator(int maxDoc, PackedLongValues values) {
+    BufferedNorms(int maxDoc, PackedLongValues values) {
       this.maxDoc = maxDoc;
       this.iter = values.iterator();
       this.size = (int) values.size();
     }
     
     @Override
-    public boolean hasNext() {
-      return upto < maxDoc;
+    public int docID() {
+      return docID;
     }
 
     @Override
-    public Number next() {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
+    public int nextDoc() {
+      docID++;
+      if (docID == maxDoc) {
+        docID = NO_MORE_DOCS;
       }
-      Long value;
-      if (upto < size) {
+      if (docID < size) {
         value = iter.next();
       } else {
         value = MISSING;
       }
-      upto++;
-      return value;
+      return docID;
     }
-
+    
     @Override
-    public void remove() {
+    public int advance(int target) {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
+    public long longValue() {
+      return value;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/NumericDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValues.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValues.java
index 33ab33b..d40f56a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValues.java
@@ -14,22 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.lucene.index;
 
+import org.apache.lucene.search.DocIdSetIterator;
 
 /**
  * A per-document numeric value.
  */
-public abstract class NumericDocValues {
+public abstract class NumericDocValues extends DocIdSetIterator {
   
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected NumericDocValues() {}
 
   /**
-   * Returns the numeric value for the specified document ID.
-   * @param docID document ID to lookup
+   * Returns the numeric value for the current document ID.
    * @return numeric value
    */
-  public abstract long get(int docID);
+  public abstract long longValue();
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
index 1d28611..4dd3cd0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java
@@ -31,7 +31,8 @@ import org.apache.lucene.util.packed.PagedMutable;
  * @lucene.experimental
  */
 class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
-  
+
+  // TODO: can't this just be NumericDocValues now?  avoid boxing the long value...
   final static class Iterator extends DocValuesFieldUpdates.Iterator {
     private final int size;
     private final PagedGrowableWriter values;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
index 917af66..03fbe10 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
@@ -18,8 +18,6 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.util.Counter;
@@ -88,57 +86,70 @@ class NumericDocValuesWriter extends DocValuesWriter {
     final PackedLongValues values = pending.build();
 
     dvConsumer.addNumericField(fieldInfo,
-                               new Iterable<Number>() {
+                               new EmptyDocValuesProducer() {
                                  @Override
-                                 public Iterator<Number> iterator() {
-                                   return new NumericIterator(maxDoc, values, docsWithField);
+                                 public NumericDocValues getNumeric(FieldInfo fieldInfo) {
+                                   if (fieldInfo != NumericDocValuesWriter.this.fieldInfo) {
+                                     throw new IllegalArgumentException("wrong fieldInfo");
+                                   }
+                                   return new BufferedNumericDocValues(maxDoc, values, docsWithField);
                                  }
                                });
   }
 
   // iterates over the values we have in ram
-  private static class NumericIterator implements Iterator<Number> {
+  private static class BufferedNumericDocValues extends NumericDocValues {
     final PackedLongValues.Iterator iter;
     final FixedBitSet docsWithField;
     final int size;
     final int maxDoc;
-    int upto;
+    private long value;
+    private int docID = -1;
     
-    NumericIterator(int maxDoc, PackedLongValues values, FixedBitSet docsWithFields) {
+    BufferedNumericDocValues(int maxDoc, PackedLongValues values, FixedBitSet docsWithFields) {
       this.maxDoc = maxDoc;
       this.iter = values.iterator();
       this.size = (int) values.size();
       this.docsWithField = docsWithFields;
     }
-    
+
     @Override
-    public boolean hasNext() {
-      return upto < maxDoc;
+    public int docID() {
+      return docID;
     }
 
     @Override
-    public Number next() {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
-      }
-      Long value;
-      if (upto < size) {
-        long v = iter.next();
-        if (docsWithField.get(upto)) {
-          value = v;
+    public int nextDoc() {
+      if (docID == size-1) {
+        docID = NO_MORE_DOCS;
+      } else {
+        int next = docsWithField.nextSetBit(docID+1);
+        if (next == NO_MORE_DOCS) {
+          docID = NO_MORE_DOCS;
         } else {
-          value = null;
+          // skip missing values:
+          while (docID < next) {
+            docID++;
+            value = iter.next();
+          }
         }
-      } else {
-        value = null;
       }
-      upto++;
-      return value;
+      return docID;
     }
-
+    
     @Override
-    public void remove() {
+    public int advance(int target) {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public long cost() {
+      return docsWithField.cardinality();
+    }
+
+    @Override
+    public long longValue() {
+      return value;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
index d85ff2d..f4afa11 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java
@@ -290,7 +290,7 @@ public class ParallelLeafReader extends LeafReader {
     LeafReader reader = fieldToReader.get(field);
     return reader == null ? null : reader.getBinaryDocValues(field);
   }
-  
+
   @Override
   public SortedDocValues getSortedDocValues(String field) throws IOException {
     ensureOpen();
@@ -313,13 +313,6 @@ public class ParallelLeafReader extends LeafReader {
   }
 
   @Override
-  public Bits getDocsWithField(String field) throws IOException {
-    ensureOpen();
-    LeafReader reader = fieldToReader.get(field);
-    return reader == null ? null : reader.getDocsWithField(field);
-  }
-
-  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     LeafReader reader = fieldToReader.get(field);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/RandomAccessOrds.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/RandomAccessOrds.java b/lucene/core/src/java/org/apache/lucene/index/RandomAccessOrds.java
deleted file mode 100644
index d0494f5..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/RandomAccessOrds.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.index;
-
-
-/** 
- * Extension of {@link SortedSetDocValues} that supports random access
- * to the ordinals of a document.
- * <p>
- * Operations via this API are independent of the iterator api ({@link #nextOrd()})
- * and do not impact its state.
- * <p>
- * Codecs can optionally extend this API if they support constant-time access
- * to ordinals for the document.
- */
-public abstract class RandomAccessOrds extends SortedSetDocValues {
-  
-  /** Sole constructor. (For invocation by subclass 
-   * constructors, typically implicit.) */
-  protected RandomAccessOrds() {}
-
-  /** 
-   * Retrieve the ordinal for the current document (previously
-   * set by {@link #setDocument(int)} at the specified index.
-   * <p>
-   * An index ranges from {@code 0} to {@code cardinality()-1}.
-   * The first ordinal value is at index {@code 0}, the next at index {@code 1},
-   * and so on, as for array indexing.
-   * @param index index of the ordinal for the document.
-   * @return ordinal for the document at the specified index.
-   */
-  public abstract long ordAt(int index);
-  
-  /** 
-   * Returns the cardinality for the current document (previously
-   * set by {@link #setDocument(int)}.
-   */
-  public abstract int cardinality();
-}