You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mv...@apache.org on 2016/03/15 10:02:17 UTC

[1/3] lucene-solr git commit: LUCENE-7091: Added doc values support to memory index

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6_0 118b8965c -> bd0803fd4
  refs/heads/branch_6x 13ce0815a -> 559432fcd
  refs/heads/master 8185c8a11 -> cf3eea264


LUCENE-7091: Added doc values support to memory index


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/cf3eea26
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/cf3eea26
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/cf3eea26

Branch: refs/heads/master
Commit: cf3eea26406692306505d2606d7ff73ee3634c30
Parents: 8185c8a
Author: Martijn van Groningen <ma...@gmail.com>
Authored: Wed Mar 9 17:38:23 2016 +0100
Committer: Martijn van Groningen <mv...@apache.org>
Committed: Tue Mar 15 09:57:40 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/index/memory/MemoryIndex.java | 465 +++++++++++++++----
 .../lucene/index/memory/TestMemoryIndex.java    | 133 ++++++
 .../memory/TestMemoryIndexAgainstRAMDir.java    | 134 ++++++
 4 files changed, 636 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf3eea26/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 10d4d10..db08eb3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -203,6 +203,9 @@ Other
 * LUCENE-7087: Let MemoryIndex#fromDocument(...) accept 'Iterable<? extends IndexableField>'
   as document instead of 'Document'. (Martijn van Groningen)
 
+* LUCENE-7091: Add doc values support to MemoryIndex
+  (Martijn van Groningen, David Smiley)
+
 ======================= Lucene 5.5.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf3eea26/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 9e01182..40159aa 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
@@ -17,12 +17,15 @@
 package org.apache.lucene.index.memory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.Locale;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -255,7 +258,7 @@ public class MemoryIndex {
       throw new IllegalArgumentException("analyzer must not be null");
     
     TokenStream stream = analyzer.tokenStream(fieldName, text);
-    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName));
+    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName), DocValuesType.NONE, null);
   }
 
   /**
@@ -351,7 +354,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @throws IllegalArgumentException if the field is a DocValues or Point field, as these
@@ -362,7 +367,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @param boost a field boost
@@ -370,17 +377,42 @@ public class MemoryIndex {
    *                                  structures are not supported by MemoryIndex
    */
   public void addField(IndexableField field, Analyzer analyzer, float boost) {
-    if (field.fieldType().docValuesType() != DocValuesType.NONE)
-      throw new IllegalArgumentException("MemoryIndex does not support DocValues fields");
-    if (field.fieldType().pointDimensionCount() != 0)
+    if (field.fieldType().pointDimensionCount() != 0) {
       throw new IllegalArgumentException("MemoryIndex does not support Points");
-    if (analyzer == null) {
-      addField(field.name(), field.tokenStream(null, null), boost);
     }
-    else {
-      addField(field.name(), field.tokenStream(analyzer, null), boost,
-          analyzer.getPositionIncrementGap(field.name()), analyzer.getOffsetGap(field.name()));
+
+    int offsetGap;
+    TokenStream tokenStream;
+    int positionIncrementGap;
+    if (analyzer != null) {
+      offsetGap = analyzer.getOffsetGap(field.name());
+      tokenStream = field.tokenStream(analyzer, null);
+      positionIncrementGap = analyzer.getPositionIncrementGap(field.name());
+    } else {
+      offsetGap = 1;
+      tokenStream = field.tokenStream(null, null);
+      positionIncrementGap = 0;
+    }
+
+    DocValuesType docValuesType = field.fieldType().docValuesType();
+    Object docValuesValue;
+    switch (docValuesType) {
+      case NONE:
+        docValuesValue = null;
+        break;
+      case BINARY:
+      case SORTED:
+      case SORTED_SET:
+        docValuesValue = field.binaryValue();
+        break;
+      case NUMERIC:
+      case SORTED_NUMERIC:
+        docValuesValue = field.numericValue();
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
     }
+    addField(field.name(), tokenStream, boost, positionIncrementGap, offsetGap, docValuesType, docValuesValue);
   }
   
   /**
@@ -448,66 +480,126 @@ public class MemoryIndex {
    *            the offset gap if fields with the same name are added more than once
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
-  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap,
-                       int offsetGap) {
-    try (TokenStream stream = tokenStream) {
-      if (frozen)
-        throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
-      if (fieldName == null)
-        throw new IllegalArgumentException("fieldName must not be null");
-      if (stream == null)
-        throw new IllegalArgumentException("token stream must not be null");
-      if (boost <= 0.0f)
-        throw new IllegalArgumentException("boost factor must be greater than 0.0");
-      int numTokens = 0;
-      int numOverlapTokens = 0;
-      int pos = -1;
-      final BytesRefHash terms;
-      final SliceByteStartArray sliceArray;
-      Info info;
-      long sumTotalTermFreq = 0;
-      int offset = 0;
-      FieldInfo fieldInfo;
-      if ((info = fields.get(fieldName)) != null) {
-        fieldInfo = info.fieldInfo;
-        numTokens = info.numTokens;
-        numOverlapTokens = info.numOverlapTokens;
-        pos = info.lastPosition + positionIncrementGap;
-        offset = info.lastOffset + offsetGap;
-        terms = info.terms;
-        boost *= info.boost;
-        sliceArray = info.sliceArray;
-        sumTotalTermFreq = info.sumTotalTermFreq;
-      } else {
-        fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, this.storePayloads,
-                                  this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
-                                  DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0);
-        sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
-        terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);
-      }
+  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    addField(fieldName, tokenStream, boost, positionIncrementGap, offsetGap, DocValuesType.NONE, null);
+  }
+
+  private void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap,
+                        DocValuesType docValuesType, Object docValuesValue) {
+
+    if (frozen) {
+      throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
+    }
+    if (fieldName == null) {
+      throw new IllegalArgumentException("fieldName must not be null");
+    }
+    if (boost <= 0.0f) {
+      throw new IllegalArgumentException("boost factor must be greater than 0.0");
+    }
+
+    Info info = fields.get(fieldName);
+    if (info == null) {
+      IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+      FieldInfo fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, storePayloads, indexOptions, docValuesType, -1, Collections.emptyMap(), 0, 0);
+      fields.put(fieldName, info = new Info(fieldInfo, byteBlockPool));
+    }
+
+    if (docValuesType != DocValuesType.NONE) {
+      storeDocValues(info, docValuesType, docValuesValue);
+    }
+    if (tokenStream != null) {
+      storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
+    }
+  }
 
+  private void storeDocValues(Info info, DocValuesType docValuesType, Object docValuesValue) {
+    String fieldName = info.fieldInfo.name;
+    DocValuesType existingDocValuesType = info.fieldInfo.getDocValuesType();
+    if (existingDocValuesType == DocValuesType.NONE) {
+      // first time we add doc values for this field:
+      info.fieldInfo = new FieldInfo(
+          info.fieldInfo.name, info.fieldInfo.number, info.fieldInfo.hasVectors(), info.fieldInfo.hasPayloads(),
+          info.fieldInfo.hasPayloads(), info.fieldInfo.getIndexOptions(), docValuesType, -1, info.fieldInfo.attributes(),
+          info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes()
+      );
+    } else if (existingDocValuesType != docValuesType) {
+      throw new IllegalArgumentException("Can't add [" + docValuesType + "] doc values field [" + fieldName + "], because [" + existingDocValuesType + "] doc values field already exists");
+    }
+    switch (docValuesType) {
+      case NUMERIC:
+        if (info.numericProducer.dvLongValues != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.numericProducer.dvLongValues = new long[]{(long) docValuesValue};
+        info.numericProducer.count++;
+        break;
+      case SORTED_NUMERIC:
+        if (info.numericProducer.dvLongValues == null) {
+          info.numericProducer.dvLongValues = new long[4];
+        }
+        info.numericProducer.dvLongValues = ArrayUtil.grow(info.numericProducer.dvLongValues, info.numericProducer.count + 1);
+        info.numericProducer.dvLongValues[info.numericProducer.count++] = (long) docValuesValue;
+        break;
+      case BINARY:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED_SET:
+        if (info.binaryProducer.dvBytesValuesSet == null) {
+          info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        }
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
+    }
+  }
+
+  private void storeTerms(Info info, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    int pos = -1;
+    int offset = 0;
+    if (info.numTokens == 0) {
+      info.boost = boost;
+    } else if (info.numTokens > 0) {
+      pos = info.lastPosition + positionIncrementGap;
+      offset = info.lastOffset + offsetGap;
+      info.boost *= boost;
+    }
+
+    try (TokenStream stream = tokenStream) {
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
       OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
       PayloadAttribute payloadAtt = storePayloads ? stream.addAttribute(PayloadAttribute.class) : null;
       stream.reset();
-      
+
       while (stream.incrementToken()) {
 //        if (DEBUG) System.err.println("token='" + term + "'");
-        numTokens++;
+        info.numTokens++;
         final int posIncr = posIncrAttribute.getPositionIncrement();
-        if (posIncr == 0)
-          numOverlapTokens++;
+        if (posIncr == 0) {
+          info.numOverlapTokens++;
+        }
         pos += posIncr;
-        int ord = terms.add(termAtt.getBytesRef());
+        int ord = info.terms.add(termAtt.getBytesRef());
         if (ord < 0) {
           ord = (-ord) - 1;
-          postingsWriter.reset(sliceArray.end[ord]);
+          postingsWriter.reset(info.sliceArray.end[ord]);
         } else {
-          sliceArray.start[ord] = postingsWriter.startNewSlice();
+          info.sliceArray.start[ord] = postingsWriter.startNewSlice();
         }
-        sliceArray.freq[ord]++;
-        sumTotalTermFreq++;
+        info.sliceArray.freq[ord]++;
+        info.sumTotalTermFreq++;
         postingsWriter.writeInt(pos);
         if (storeOffsets) {
           postingsWriter.writeInt(offsetAtt.startOffset() + offset);
@@ -523,13 +615,12 @@ public class MemoryIndex {
           }
           postingsWriter.writeInt(pIndex);
         }
-        sliceArray.end[ord] = postingsWriter.getCurrentOffset();
+        info.sliceArray.end[ord] = postingsWriter.getCurrentOffset();
       }
       stream.end();
-
-      // ensure infos.numTokens > 0 invariant; needed for correct operation of terms()
-      if (numTokens > 0) {
-        fields.put(fieldName, new Info(fieldInfo, terms, sliceArray, numTokens, numOverlapTokens, boost, pos, offsetAtt.endOffset() + offset, sumTotalTermFreq));
+      if (info.numTokens > 0) {
+        info.lastPosition = pos;
+        info.lastOffset = offsetAtt.endOffset() + offset;
       }
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -573,8 +664,7 @@ public class MemoryIndex {
   public void freeze() {
     this.frozen = true;
     for (Info info : fields.values()) {
-      info.sortTerms();
-      info.getNormDocValues();//lazily computed
+      info.freeze();
     }
   }
   
@@ -702,7 +792,7 @@ public class MemoryIndex {
    */
   private final class Info {
 
-    private final FieldInfo fieldInfo;
+    private FieldInfo fieldInfo;
 
     /** The norms for this field; computed on demand. */
     private transient NumericDocValues norms;
@@ -711,40 +801,48 @@ public class MemoryIndex {
      * Term strings and their positions for this field: Map &lt;String
      * termText, ArrayIntList positions&gt;
      */
-    private final BytesRefHash terms; // note unfortunate variable name class with Terms type
+    private BytesRefHash terms; // note unfortunate variable name class with Terms type
     
-    private final SliceByteStartArray sliceArray;
+    private SliceByteStartArray sliceArray;
 
     /** Terms sorted ascending by term text; computed on demand */
     private transient int[] sortedTerms;
     
     /** Number of added tokens for this field */
-    private final int numTokens;
+    private int numTokens;
     
     /** Number of overlapping tokens for this field */
-    private final int numOverlapTokens;
+    private int numOverlapTokens;
     
     /** Boost factor for hits for this field */
-    private final float boost;
+    private float boost;
 
-    private final long sumTotalTermFreq;
+    private long sumTotalTermFreq;
 
     /** the last position encountered in this field for multi field support*/
-    private final int lastPosition;
+    private int lastPosition;
 
     /** the last offset encountered in this field for multi field support*/
-    private final int lastOffset;
+    private int lastOffset;
 
-    public Info(FieldInfo fieldInfo, BytesRefHash terms, SliceByteStartArray sliceArray, int numTokens, int numOverlapTokens, float boost, int lastPosition, int lastOffset, long sumTotalTermFreq) {
+    private BinaryDocValuesProducer binaryProducer;
+
+    private NumericDocValuesProducer numericProducer;
+
+    private boolean preparedDocValues;
+
+    private Info(FieldInfo fieldInfo, ByteBlockPool byteBlockPool) {
       this.fieldInfo = fieldInfo;
-      this.terms = terms;
-      this.sliceArray = sliceArray; 
-      this.numTokens = numTokens;
-      this.numOverlapTokens = numOverlapTokens;
-      this.boost = boost;
-      this.sumTotalTermFreq = sumTotalTermFreq;
-      this.lastPosition = lastPosition;
-      this.lastOffset = lastOffset;
+      this.sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
+      this.terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);;
+      this.binaryProducer = new BinaryDocValuesProducer();
+      this.numericProducer = new NumericDocValuesProducer();
+    }
+
+    void freeze() {
+      sortTerms();
+      prepareDocValues();
+      getNormDocValues();
     }
 
     /**
@@ -755,13 +853,26 @@ public class MemoryIndex {
      * (which would be an alternative and somewhat more elegant approach,
      * apart from more sophisticated Tries / prefix trees).
      */
-    public void sortTerms() {
+    void sortTerms() {
       if (sortedTerms == null) {
         sortedTerms = terms.sort();
       }
     }
 
-    public NumericDocValues getNormDocValues() {
+    void prepareDocValues() {
+      if (preparedDocValues == false) {
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        if (dvType == DocValuesType.NUMERIC || dvType == DocValuesType.SORTED_NUMERIC) {
+          numericProducer.prepareForUsage();
+        }
+        if (dvType == DocValuesType.BINARY || dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
+          binaryProducer.prepareForUsage();
+        }
+        preparedDocValues = true;
+      }
+    }
+
+    NumericDocValues getNormDocValues() {
       if (norms == null) {
         FieldInvertState invertState = new FieldInvertState(fieldInfo.name, fieldInfo.number,
             numTokens, numOverlapTokens, 0, boost);
@@ -786,7 +897,81 @@ public class MemoryIndex {
   ///////////////////////////////////////////////////////////////////////////////
   // Nested classes:
   ///////////////////////////////////////////////////////////////////////////////
-    
+
+  private static final class BinaryDocValuesProducer {
+
+    BytesRefHash dvBytesValuesSet;
+    final SortedDocValues sortedDocValues;
+    final BytesRef spare = new BytesRef();
+
+    int[] bytesIds;
+
+    private BinaryDocValuesProducer() {
+      sortedDocValues = new SortedDocValues() {
+        @Override
+        public int getOrd(int docID) {
+          return 0;
+        }
+
+        @Override
+        public BytesRef lookupOrd(int ord) {
+          return getValue(ord);
+        }
+
+        @Override
+        public int getValueCount() {
+          return 1;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      bytesIds = dvBytesValuesSet.sort();
+    }
+
+    private BytesRef getValue(int index) {
+      return dvBytesValuesSet.get(bytesIds[index], spare);
+    }
+
+  }
+
+  private static final class NumericDocValuesProducer {
+
+    long[] dvLongValues;
+    int count;
+
+    final NumericDocValues numericDocValues;
+    final SortedNumericDocValues sortedNumericDocValues;
+
+    private NumericDocValuesProducer() {
+      this.numericDocValues = new NumericDocValues() {
+        @Override
+        public long get(int docID) {
+          return dvLongValues[0];
+        }
+      };
+      this.sortedNumericDocValues = new SortedNumericDocValues() {
+        @Override
+        public void setDocument(int doc) {
+        }
+
+        @Override
+        public long valueAt(int index) {
+          return dvLongValues[index];
+        }
+
+        @Override
+        public int count() {
+          return count;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      Arrays.sort(dvLongValues, 0, count);
+    }
+  }
+
   /**
    * Search support for Lucene framework integration; implements all methods
    * required by the Lucene IndexReader contracts.
@@ -795,6 +980,9 @@ public class MemoryIndex {
     
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
+      for (Info info : fields.values()) {
+        info.prepareDocValues();
+      }
     }
 
     @Override
@@ -807,8 +995,18 @@ public class MemoryIndex {
       removeCoreClosedListenerAsReaderClosedListener(this, listener);
     }
 
-    private Info getInfo(String fieldName) {
-      return fields.get(fieldName);
+    private Info getInfoForExpectedDocValuesType(String fieldName, DocValuesType expectedType) {
+      if (expectedType == DocValuesType.NONE) {
+        return null;
+      }
+      Info info = fields.get(fieldName);
+      if (info == null) {
+        return null;
+      }
+      if (info.fieldInfo.getDocValuesType() != expectedType) {
+        return null;
+      }
+      return info;
     }
 
     @Override
@@ -828,32 +1026,87 @@ public class MemoryIndex {
 
     @Override
     public NumericDocValues getNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.NUMERIC);
+      if (info != null) {
+        return info.numericProducer.numericDocValues;
+      } else {
+        return null;
+      }
     }
 
     @Override
     public BinaryDocValues getBinaryDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.BINARY);
     }
 
     @Override
     public SortedDocValues getSortedDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.SORTED);
+    }
+
+    private SortedDocValues getSortedDocValues(String field, DocValuesType docValuesType) {
+      Info info = getInfoForExpectedDocValuesType(field, docValuesType);
+      if (info != null) {
+        return info.binaryProducer.sortedDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedNumericDocValues getSortedNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_NUMERIC);
+      if (info != null) {
+        return info.numericProducer.sortedNumericDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedSetDocValues getSortedSetDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_SET);
+      if (info != null) {
+        return new SortedSetDocValues() {
+
+          int index = 0;
+
+          @Override
+          public long nextOrd() {
+            if (index >= info.binaryProducer.dvBytesValuesSet.size()) {
+              return NO_MORE_ORDS;
+            }
+            return index++;
+          }
+
+          @Override
+          public void setDocument(int docID) {
+            index = 0;
+          }
+
+          @Override
+          public BytesRef lookupOrd(long ord) {
+            return info.binaryProducer.getValue((int) ord);
+          }
+
+          @Override
+          public long getValueCount() {
+            return info.binaryProducer.dvBytesValuesSet.size();
+          }
+        };
+      } else {
+        return null;
+      }
     }
 
     @Override
     public Bits getDocsWithField(String field) throws IOException {
-      return null;
+      Info info = fields.get(field);
+      if (info != null && info.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+        return new Bits.MatchAllBits(1);
+      } else {
+        return null;
+      }
     }
 
     @Override
@@ -866,7 +1119,25 @@ public class MemoryIndex {
       // no-op
     }
 
+    @Override
+    public Fields fields() {
+      Map<String, Info> filteredFields = fields.entrySet().stream()
+          .filter(entry ->  entry.getValue().numTokens > 0)
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
+              (u,v) -> { throw new IllegalStateException(String.format(Locale.ROOT, "Duplicate key %s", u));},
+              TreeMap::new
+          ));
+      return new MemoryFields(filteredFields );
+    }
+
     private class MemoryFields extends Fields {
+
+      private final Map<String, Info> fields;
+
+      public MemoryFields(Map<String, Info> fields) {
+        this.fields = fields;
+      }
+
       @Override
       public Iterator<String> iterator() {
         return fields.keySet().iterator();
@@ -875,8 +1146,9 @@ public class MemoryIndex {
       @Override
       public Terms terms(final String field) {
         final Info info = fields.get(field);
-        if (info == null)
+        if (info == null) {
           return null;
+        }
 
         return new Terms() {
           @Override
@@ -932,11 +1204,6 @@ public class MemoryIndex {
         return fields.size();
       }
     }
-  
-    @Override
-    public Fields fields() {
-      return new MemoryFields();
-    }
 
     private class MemoryTermsEnum extends TermsEnum {
       private final Info info;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf3eea26/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 b150ea3..7282e0e 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
@@ -21,13 +21,25 @@ import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockPayloadAnalyzer;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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.FieldInvertState;
 import org.apache.lucene.index.LeafReader;
+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.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -35,6 +47,7 @@ import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Before;
@@ -176,5 +189,125 @@ public class TestMemoryIndex extends LuceneTestCase {
 
   }
 
+  public void testDocValues() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("numeric", 29L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 33L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 31L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 30L));
+    doc.add(new BinaryDocValuesField("binary", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("sorted", new BytesRef("b")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("f")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("c")));
+
+    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));
+    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));
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    assertEquals("a", binaryDocValues.get(0).utf8ToString());
+    SortedDocValues sortedDocValues = leafReader.getSortedDocValues("sorted");
+    assertEquals("b", sortedDocValues.get(0).utf8ToString());
+    assertEquals(0, sortedDocValues.getOrd(0));
+    assertEquals("b", sortedDocValues.lookupOrd(0).utf8ToString());
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    assertEquals(3, sortedSetDocValues.getValueCount());
+    sortedSetDocValues.setDocument(0);
+    assertEquals(0L, sortedSetDocValues.nextOrd());
+    assertEquals(1L, sortedSetDocValues.nextOrd());
+    assertEquals(2L, sortedSetDocValues.nextOrd());
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+    assertEquals("c", sortedSetDocValues.lookupOrd(0L).utf8ToString());
+    assertEquals("d", sortedSetDocValues.lookupOrd(1L).utf8ToString());
+    assertEquals("f", sortedSetDocValues.lookupOrd(2L).utf8ToString());
+  }
+
+  public void testInvalidDocValuesUsage() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("30")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Can't add [BINARY] doc values field [field], because [NUMERIC] doc values field already exists", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new NumericDocValuesField("field", 30L));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [NUMERIC] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("a")));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("b")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [BINARY] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new SortedDocValuesField("field", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("field", new BytesRef("b")));
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [SORTED] doc values field [field]", e.getMessage());
+    }
+  }
+
+  public void testDocValuesDoNotAffectBoostPositionsOrOffset() throws Exception {
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    doc.add(new TextField("text", "quick brown fox", Field.Store.NO));
+    MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer, true, true);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+    TermsEnum tenum = leafReader.terms("text").iterator();
+
+    assertEquals("brown", tenum.next().utf8ToString());
+    PostingsEnum penum = tenum.postings(null, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(1, penum.nextPosition());
+    assertEquals(6, penum.startOffset());
+    assertEquals(11, penum.endOffset());
+
+    assertEquals("fox", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(2, penum.nextPosition());
+    assertEquals(12, penum.startOffset());
+    assertEquals(15, penum.endOffset());
+
+    assertEquals("quick", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(0, penum.nextPosition());
+    assertEquals(0, penum.startOffset());
+    assertEquals(5, penum.endOffset());
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("text");
+    assertEquals("quick brown fox", binaryDocValues.get(0).utf8ToString());
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cf3eea26/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 57e25fe..3e6778a 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
@@ -35,10 +35,18 @@ import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.LegacyLongField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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;
@@ -52,6 +60,9 @@ import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiFields;
 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;
@@ -434,6 +445,129 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
     assertNull(reader.terms("not-in-index"));
   }
 
+  public void testDocValuesMemoryIndexVsNormalIndex() throws Exception {
+    Document doc = new Document();
+    long randomLong = random().nextLong();
+    doc.add(new NumericDocValuesField("numeric", randomLong));
+    if (random().nextBoolean()) {
+      doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+    }
+    int numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomLong = random().nextLong();
+      doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      }
+      if (random().nextBoolean()) {
+        doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+      }
+    }
+    BytesRef randomTerm = new BytesRef(randomTerm());
+    doc.add(new BinaryDocValuesField("binary", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("binary", randomTerm, Field.Store.NO));
+    }
+    randomTerm = new BytesRef(randomTerm());
+    doc.add(new SortedDocValuesField("sorted", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("sorted", randomTerm, Field.Store.NO));
+    }
+    numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomTerm = new BytesRef(randomTerm());
+      doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      }
+      if (random().nextBoolean()) {
+        // randomily just add a normal string field
+        doc.add(new StringField("sorted_set", randomTerm, Field.Store.NO));
+      }
+    }
+
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+    MemoryIndex memoryIndex = MemoryIndex.fromDocument(doc, mockAnalyzer);
+    IndexReader indexReader = memoryIndex.createSearcher().getIndexReader();
+    LeafReader leafReader =  indexReader.leaves().get(0).reader();
+
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    NumericDocValues numericDocValues = leafReader.getNumericDocValues("numeric");
+    NumericDocValues controlNumericDocValues = controlLeafReader.getNumericDocValues("numeric");
+    assertEquals(controlNumericDocValues.get(0), numericDocValues.get(0));
+
+    SortedNumericDocValues sortedNumericDocValues = leafReader.getSortedNumericDocValues("sorted_numeric");
+    sortedNumericDocValues.setDocument(0);
+    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));
+    }
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    BinaryDocValues controlBinaryDocValues = controlLeafReader.getBinaryDocValues("binary");
+    assertEquals(controlBinaryDocValues.get(0), binaryDocValues.get(0));
+
+    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(controlSortedDocValues.lookupOrd(0), sortedDocValues.lookupOrd(0));
+
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    sortedSetDocValues.setDocument(0);
+    SortedSetDocValues controlSortedSetDocValues = controlLeafReader.getSortedSetDocValues("sorted_set");
+    controlSortedSetDocValues.setDocument(0);
+    assertEquals(controlSortedSetDocValues.getValueCount(), sortedSetDocValues.getValueCount());
+    for (long controlOrd = controlSortedSetDocValues.nextOrd(); controlOrd != SortedSetDocValues.NO_MORE_ORDS;
+         controlOrd = controlSortedSetDocValues.nextOrd()) {
+      assertEquals(controlOrd, sortedSetDocValues.nextOrd());
+      assertEquals(controlSortedSetDocValues.lookupOrd(controlOrd), sortedSetDocValues.lookupOrd(controlOrd));
+    }
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+
+    indexReader.close();
+    controlIndexReader.close();
+    dir.close();
+  }
+
+  public void testNormsWithDocValues() throws Exception {
+    MemoryIndex mi = new MemoryIndex(true, true);
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+
+    mi.addField(new BinaryDocValuesField("text", new BytesRef("quick brown fox")), mockAnalyzer, 5f);
+    mi.addField(new TextField("text", "quick brown fox", Field.Store.NO), mockAnalyzer, 5f);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    Field field = new TextField("text", "quick brown fox", Field.Store.NO);
+    field.setBoost(5f);
+    doc.add(field);
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    assertEquals(controlLeafReader.getNormValues("text").get(0), leafReader.getNormValues("text").get(0));
+
+    controlIndexReader.close();
+    dir.close();
+  }
+
   public void testDuellMemIndex() throws IOException {
     LineFileDocs lineFileDocs = new LineFileDocs(random());
     int numDocs = atLeast(10);


[2/3] lucene-solr git commit: LUCENE-7091: Added doc values support to memory index

Posted by mv...@apache.org.
LUCENE-7091: Added doc values support to memory index


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/559432fc
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/559432fc
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/559432fc

Branch: refs/heads/branch_6x
Commit: 559432fcdcd828864a800072cf144e1f5c96647e
Parents: 13ce081
Author: Martijn van Groningen <ma...@gmail.com>
Authored: Wed Mar 9 17:38:23 2016 +0100
Committer: Martijn van Groningen <mv...@apache.org>
Committed: Tue Mar 15 10:00:19 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/index/memory/MemoryIndex.java | 465 +++++++++++++++----
 .../lucene/index/memory/TestMemoryIndex.java    | 133 ++++++
 .../memory/TestMemoryIndexAgainstRAMDir.java    | 134 ++++++
 4 files changed, 636 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/559432fc/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 96321b0..0897c1a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -196,6 +196,9 @@ Other
 * LUCENE-7087: Let MemoryIndex#fromDocument(...) accept 'Iterable<? extends IndexableField>'
   as document instead of 'Document'. (Martijn van Groningen)
 
+* LUCENE-7091: Add doc values support to MemoryIndex
+  (Martijn van Groningen, David Smiley)
+
 ======================= Lucene 5.5.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/559432fc/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 9e01182..40159aa 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
@@ -17,12 +17,15 @@
 package org.apache.lucene.index.memory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.Locale;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -255,7 +258,7 @@ public class MemoryIndex {
       throw new IllegalArgumentException("analyzer must not be null");
     
     TokenStream stream = analyzer.tokenStream(fieldName, text);
-    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName));
+    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName), DocValuesType.NONE, null);
   }
 
   /**
@@ -351,7 +354,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @throws IllegalArgumentException if the field is a DocValues or Point field, as these
@@ -362,7 +367,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @param boost a field boost
@@ -370,17 +377,42 @@ public class MemoryIndex {
    *                                  structures are not supported by MemoryIndex
    */
   public void addField(IndexableField field, Analyzer analyzer, float boost) {
-    if (field.fieldType().docValuesType() != DocValuesType.NONE)
-      throw new IllegalArgumentException("MemoryIndex does not support DocValues fields");
-    if (field.fieldType().pointDimensionCount() != 0)
+    if (field.fieldType().pointDimensionCount() != 0) {
       throw new IllegalArgumentException("MemoryIndex does not support Points");
-    if (analyzer == null) {
-      addField(field.name(), field.tokenStream(null, null), boost);
     }
-    else {
-      addField(field.name(), field.tokenStream(analyzer, null), boost,
-          analyzer.getPositionIncrementGap(field.name()), analyzer.getOffsetGap(field.name()));
+
+    int offsetGap;
+    TokenStream tokenStream;
+    int positionIncrementGap;
+    if (analyzer != null) {
+      offsetGap = analyzer.getOffsetGap(field.name());
+      tokenStream = field.tokenStream(analyzer, null);
+      positionIncrementGap = analyzer.getPositionIncrementGap(field.name());
+    } else {
+      offsetGap = 1;
+      tokenStream = field.tokenStream(null, null);
+      positionIncrementGap = 0;
+    }
+
+    DocValuesType docValuesType = field.fieldType().docValuesType();
+    Object docValuesValue;
+    switch (docValuesType) {
+      case NONE:
+        docValuesValue = null;
+        break;
+      case BINARY:
+      case SORTED:
+      case SORTED_SET:
+        docValuesValue = field.binaryValue();
+        break;
+      case NUMERIC:
+      case SORTED_NUMERIC:
+        docValuesValue = field.numericValue();
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
     }
+    addField(field.name(), tokenStream, boost, positionIncrementGap, offsetGap, docValuesType, docValuesValue);
   }
   
   /**
@@ -448,66 +480,126 @@ public class MemoryIndex {
    *            the offset gap if fields with the same name are added more than once
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
-  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap,
-                       int offsetGap) {
-    try (TokenStream stream = tokenStream) {
-      if (frozen)
-        throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
-      if (fieldName == null)
-        throw new IllegalArgumentException("fieldName must not be null");
-      if (stream == null)
-        throw new IllegalArgumentException("token stream must not be null");
-      if (boost <= 0.0f)
-        throw new IllegalArgumentException("boost factor must be greater than 0.0");
-      int numTokens = 0;
-      int numOverlapTokens = 0;
-      int pos = -1;
-      final BytesRefHash terms;
-      final SliceByteStartArray sliceArray;
-      Info info;
-      long sumTotalTermFreq = 0;
-      int offset = 0;
-      FieldInfo fieldInfo;
-      if ((info = fields.get(fieldName)) != null) {
-        fieldInfo = info.fieldInfo;
-        numTokens = info.numTokens;
-        numOverlapTokens = info.numOverlapTokens;
-        pos = info.lastPosition + positionIncrementGap;
-        offset = info.lastOffset + offsetGap;
-        terms = info.terms;
-        boost *= info.boost;
-        sliceArray = info.sliceArray;
-        sumTotalTermFreq = info.sumTotalTermFreq;
-      } else {
-        fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, this.storePayloads,
-                                  this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
-                                  DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0);
-        sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
-        terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);
-      }
+  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    addField(fieldName, tokenStream, boost, positionIncrementGap, offsetGap, DocValuesType.NONE, null);
+  }
+
+  private void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap,
+                        DocValuesType docValuesType, Object docValuesValue) {
+
+    if (frozen) {
+      throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
+    }
+    if (fieldName == null) {
+      throw new IllegalArgumentException("fieldName must not be null");
+    }
+    if (boost <= 0.0f) {
+      throw new IllegalArgumentException("boost factor must be greater than 0.0");
+    }
+
+    Info info = fields.get(fieldName);
+    if (info == null) {
+      IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+      FieldInfo fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, storePayloads, indexOptions, docValuesType, -1, Collections.emptyMap(), 0, 0);
+      fields.put(fieldName, info = new Info(fieldInfo, byteBlockPool));
+    }
+
+    if (docValuesType != DocValuesType.NONE) {
+      storeDocValues(info, docValuesType, docValuesValue);
+    }
+    if (tokenStream != null) {
+      storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
+    }
+  }
 
+  private void storeDocValues(Info info, DocValuesType docValuesType, Object docValuesValue) {
+    String fieldName = info.fieldInfo.name;
+    DocValuesType existingDocValuesType = info.fieldInfo.getDocValuesType();
+    if (existingDocValuesType == DocValuesType.NONE) {
+      // first time we add doc values for this field:
+      info.fieldInfo = new FieldInfo(
+          info.fieldInfo.name, info.fieldInfo.number, info.fieldInfo.hasVectors(), info.fieldInfo.hasPayloads(),
+          info.fieldInfo.hasPayloads(), info.fieldInfo.getIndexOptions(), docValuesType, -1, info.fieldInfo.attributes(),
+          info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes()
+      );
+    } else if (existingDocValuesType != docValuesType) {
+      throw new IllegalArgumentException("Can't add [" + docValuesType + "] doc values field [" + fieldName + "], because [" + existingDocValuesType + "] doc values field already exists");
+    }
+    switch (docValuesType) {
+      case NUMERIC:
+        if (info.numericProducer.dvLongValues != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.numericProducer.dvLongValues = new long[]{(long) docValuesValue};
+        info.numericProducer.count++;
+        break;
+      case SORTED_NUMERIC:
+        if (info.numericProducer.dvLongValues == null) {
+          info.numericProducer.dvLongValues = new long[4];
+        }
+        info.numericProducer.dvLongValues = ArrayUtil.grow(info.numericProducer.dvLongValues, info.numericProducer.count + 1);
+        info.numericProducer.dvLongValues[info.numericProducer.count++] = (long) docValuesValue;
+        break;
+      case BINARY:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED_SET:
+        if (info.binaryProducer.dvBytesValuesSet == null) {
+          info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        }
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
+    }
+  }
+
+  private void storeTerms(Info info, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    int pos = -1;
+    int offset = 0;
+    if (info.numTokens == 0) {
+      info.boost = boost;
+    } else if (info.numTokens > 0) {
+      pos = info.lastPosition + positionIncrementGap;
+      offset = info.lastOffset + offsetGap;
+      info.boost *= boost;
+    }
+
+    try (TokenStream stream = tokenStream) {
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
       OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
       PayloadAttribute payloadAtt = storePayloads ? stream.addAttribute(PayloadAttribute.class) : null;
       stream.reset();
-      
+
       while (stream.incrementToken()) {
 //        if (DEBUG) System.err.println("token='" + term + "'");
-        numTokens++;
+        info.numTokens++;
         final int posIncr = posIncrAttribute.getPositionIncrement();
-        if (posIncr == 0)
-          numOverlapTokens++;
+        if (posIncr == 0) {
+          info.numOverlapTokens++;
+        }
         pos += posIncr;
-        int ord = terms.add(termAtt.getBytesRef());
+        int ord = info.terms.add(termAtt.getBytesRef());
         if (ord < 0) {
           ord = (-ord) - 1;
-          postingsWriter.reset(sliceArray.end[ord]);
+          postingsWriter.reset(info.sliceArray.end[ord]);
         } else {
-          sliceArray.start[ord] = postingsWriter.startNewSlice();
+          info.sliceArray.start[ord] = postingsWriter.startNewSlice();
         }
-        sliceArray.freq[ord]++;
-        sumTotalTermFreq++;
+        info.sliceArray.freq[ord]++;
+        info.sumTotalTermFreq++;
         postingsWriter.writeInt(pos);
         if (storeOffsets) {
           postingsWriter.writeInt(offsetAtt.startOffset() + offset);
@@ -523,13 +615,12 @@ public class MemoryIndex {
           }
           postingsWriter.writeInt(pIndex);
         }
-        sliceArray.end[ord] = postingsWriter.getCurrentOffset();
+        info.sliceArray.end[ord] = postingsWriter.getCurrentOffset();
       }
       stream.end();
-
-      // ensure infos.numTokens > 0 invariant; needed for correct operation of terms()
-      if (numTokens > 0) {
-        fields.put(fieldName, new Info(fieldInfo, terms, sliceArray, numTokens, numOverlapTokens, boost, pos, offsetAtt.endOffset() + offset, sumTotalTermFreq));
+      if (info.numTokens > 0) {
+        info.lastPosition = pos;
+        info.lastOffset = offsetAtt.endOffset() + offset;
       }
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -573,8 +664,7 @@ public class MemoryIndex {
   public void freeze() {
     this.frozen = true;
     for (Info info : fields.values()) {
-      info.sortTerms();
-      info.getNormDocValues();//lazily computed
+      info.freeze();
     }
   }
   
@@ -702,7 +792,7 @@ public class MemoryIndex {
    */
   private final class Info {
 
-    private final FieldInfo fieldInfo;
+    private FieldInfo fieldInfo;
 
     /** The norms for this field; computed on demand. */
     private transient NumericDocValues norms;
@@ -711,40 +801,48 @@ public class MemoryIndex {
      * Term strings and their positions for this field: Map &lt;String
      * termText, ArrayIntList positions&gt;
      */
-    private final BytesRefHash terms; // note unfortunate variable name class with Terms type
+    private BytesRefHash terms; // note unfortunate variable name class with Terms type
     
-    private final SliceByteStartArray sliceArray;
+    private SliceByteStartArray sliceArray;
 
     /** Terms sorted ascending by term text; computed on demand */
     private transient int[] sortedTerms;
     
     /** Number of added tokens for this field */
-    private final int numTokens;
+    private int numTokens;
     
     /** Number of overlapping tokens for this field */
-    private final int numOverlapTokens;
+    private int numOverlapTokens;
     
     /** Boost factor for hits for this field */
-    private final float boost;
+    private float boost;
 
-    private final long sumTotalTermFreq;
+    private long sumTotalTermFreq;
 
     /** the last position encountered in this field for multi field support*/
-    private final int lastPosition;
+    private int lastPosition;
 
     /** the last offset encountered in this field for multi field support*/
-    private final int lastOffset;
+    private int lastOffset;
 
-    public Info(FieldInfo fieldInfo, BytesRefHash terms, SliceByteStartArray sliceArray, int numTokens, int numOverlapTokens, float boost, int lastPosition, int lastOffset, long sumTotalTermFreq) {
+    private BinaryDocValuesProducer binaryProducer;
+
+    private NumericDocValuesProducer numericProducer;
+
+    private boolean preparedDocValues;
+
+    private Info(FieldInfo fieldInfo, ByteBlockPool byteBlockPool) {
       this.fieldInfo = fieldInfo;
-      this.terms = terms;
-      this.sliceArray = sliceArray; 
-      this.numTokens = numTokens;
-      this.numOverlapTokens = numOverlapTokens;
-      this.boost = boost;
-      this.sumTotalTermFreq = sumTotalTermFreq;
-      this.lastPosition = lastPosition;
-      this.lastOffset = lastOffset;
+      this.sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
+      this.terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);;
+      this.binaryProducer = new BinaryDocValuesProducer();
+      this.numericProducer = new NumericDocValuesProducer();
+    }
+
+    void freeze() {
+      sortTerms();
+      prepareDocValues();
+      getNormDocValues();
     }
 
     /**
@@ -755,13 +853,26 @@ public class MemoryIndex {
      * (which would be an alternative and somewhat more elegant approach,
      * apart from more sophisticated Tries / prefix trees).
      */
-    public void sortTerms() {
+    void sortTerms() {
       if (sortedTerms == null) {
         sortedTerms = terms.sort();
       }
     }
 
-    public NumericDocValues getNormDocValues() {
+    void prepareDocValues() {
+      if (preparedDocValues == false) {
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        if (dvType == DocValuesType.NUMERIC || dvType == DocValuesType.SORTED_NUMERIC) {
+          numericProducer.prepareForUsage();
+        }
+        if (dvType == DocValuesType.BINARY || dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
+          binaryProducer.prepareForUsage();
+        }
+        preparedDocValues = true;
+      }
+    }
+
+    NumericDocValues getNormDocValues() {
       if (norms == null) {
         FieldInvertState invertState = new FieldInvertState(fieldInfo.name, fieldInfo.number,
             numTokens, numOverlapTokens, 0, boost);
@@ -786,7 +897,81 @@ public class MemoryIndex {
   ///////////////////////////////////////////////////////////////////////////////
   // Nested classes:
   ///////////////////////////////////////////////////////////////////////////////
-    
+
+  private static final class BinaryDocValuesProducer {
+
+    BytesRefHash dvBytesValuesSet;
+    final SortedDocValues sortedDocValues;
+    final BytesRef spare = new BytesRef();
+
+    int[] bytesIds;
+
+    private BinaryDocValuesProducer() {
+      sortedDocValues = new SortedDocValues() {
+        @Override
+        public int getOrd(int docID) {
+          return 0;
+        }
+
+        @Override
+        public BytesRef lookupOrd(int ord) {
+          return getValue(ord);
+        }
+
+        @Override
+        public int getValueCount() {
+          return 1;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      bytesIds = dvBytesValuesSet.sort();
+    }
+
+    private BytesRef getValue(int index) {
+      return dvBytesValuesSet.get(bytesIds[index], spare);
+    }
+
+  }
+
+  private static final class NumericDocValuesProducer {
+
+    long[] dvLongValues;
+    int count;
+
+    final NumericDocValues numericDocValues;
+    final SortedNumericDocValues sortedNumericDocValues;
+
+    private NumericDocValuesProducer() {
+      this.numericDocValues = new NumericDocValues() {
+        @Override
+        public long get(int docID) {
+          return dvLongValues[0];
+        }
+      };
+      this.sortedNumericDocValues = new SortedNumericDocValues() {
+        @Override
+        public void setDocument(int doc) {
+        }
+
+        @Override
+        public long valueAt(int index) {
+          return dvLongValues[index];
+        }
+
+        @Override
+        public int count() {
+          return count;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      Arrays.sort(dvLongValues, 0, count);
+    }
+  }
+
   /**
    * Search support for Lucene framework integration; implements all methods
    * required by the Lucene IndexReader contracts.
@@ -795,6 +980,9 @@ public class MemoryIndex {
     
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
+      for (Info info : fields.values()) {
+        info.prepareDocValues();
+      }
     }
 
     @Override
@@ -807,8 +995,18 @@ public class MemoryIndex {
       removeCoreClosedListenerAsReaderClosedListener(this, listener);
     }
 
-    private Info getInfo(String fieldName) {
-      return fields.get(fieldName);
+    private Info getInfoForExpectedDocValuesType(String fieldName, DocValuesType expectedType) {
+      if (expectedType == DocValuesType.NONE) {
+        return null;
+      }
+      Info info = fields.get(fieldName);
+      if (info == null) {
+        return null;
+      }
+      if (info.fieldInfo.getDocValuesType() != expectedType) {
+        return null;
+      }
+      return info;
     }
 
     @Override
@@ -828,32 +1026,87 @@ public class MemoryIndex {
 
     @Override
     public NumericDocValues getNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.NUMERIC);
+      if (info != null) {
+        return info.numericProducer.numericDocValues;
+      } else {
+        return null;
+      }
     }
 
     @Override
     public BinaryDocValues getBinaryDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.BINARY);
     }
 
     @Override
     public SortedDocValues getSortedDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.SORTED);
+    }
+
+    private SortedDocValues getSortedDocValues(String field, DocValuesType docValuesType) {
+      Info info = getInfoForExpectedDocValuesType(field, docValuesType);
+      if (info != null) {
+        return info.binaryProducer.sortedDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedNumericDocValues getSortedNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_NUMERIC);
+      if (info != null) {
+        return info.numericProducer.sortedNumericDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedSetDocValues getSortedSetDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_SET);
+      if (info != null) {
+        return new SortedSetDocValues() {
+
+          int index = 0;
+
+          @Override
+          public long nextOrd() {
+            if (index >= info.binaryProducer.dvBytesValuesSet.size()) {
+              return NO_MORE_ORDS;
+            }
+            return index++;
+          }
+
+          @Override
+          public void setDocument(int docID) {
+            index = 0;
+          }
+
+          @Override
+          public BytesRef lookupOrd(long ord) {
+            return info.binaryProducer.getValue((int) ord);
+          }
+
+          @Override
+          public long getValueCount() {
+            return info.binaryProducer.dvBytesValuesSet.size();
+          }
+        };
+      } else {
+        return null;
+      }
     }
 
     @Override
     public Bits getDocsWithField(String field) throws IOException {
-      return null;
+      Info info = fields.get(field);
+      if (info != null && info.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+        return new Bits.MatchAllBits(1);
+      } else {
+        return null;
+      }
     }
 
     @Override
@@ -866,7 +1119,25 @@ public class MemoryIndex {
       // no-op
     }
 
+    @Override
+    public Fields fields() {
+      Map<String, Info> filteredFields = fields.entrySet().stream()
+          .filter(entry ->  entry.getValue().numTokens > 0)
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
+              (u,v) -> { throw new IllegalStateException(String.format(Locale.ROOT, "Duplicate key %s", u));},
+              TreeMap::new
+          ));
+      return new MemoryFields(filteredFields );
+    }
+
     private class MemoryFields extends Fields {
+
+      private final Map<String, Info> fields;
+
+      public MemoryFields(Map<String, Info> fields) {
+        this.fields = fields;
+      }
+
       @Override
       public Iterator<String> iterator() {
         return fields.keySet().iterator();
@@ -875,8 +1146,9 @@ public class MemoryIndex {
       @Override
       public Terms terms(final String field) {
         final Info info = fields.get(field);
-        if (info == null)
+        if (info == null) {
           return null;
+        }
 
         return new Terms() {
           @Override
@@ -932,11 +1204,6 @@ public class MemoryIndex {
         return fields.size();
       }
     }
-  
-    @Override
-    public Fields fields() {
-      return new MemoryFields();
-    }
 
     private class MemoryTermsEnum extends TermsEnum {
       private final Info info;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/559432fc/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 b150ea3..7282e0e 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
@@ -21,13 +21,25 @@ import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockPayloadAnalyzer;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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.FieldInvertState;
 import org.apache.lucene.index.LeafReader;
+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.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -35,6 +47,7 @@ import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Before;
@@ -176,5 +189,125 @@ public class TestMemoryIndex extends LuceneTestCase {
 
   }
 
+  public void testDocValues() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("numeric", 29L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 33L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 31L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 30L));
+    doc.add(new BinaryDocValuesField("binary", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("sorted", new BytesRef("b")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("f")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("c")));
+
+    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));
+    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));
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    assertEquals("a", binaryDocValues.get(0).utf8ToString());
+    SortedDocValues sortedDocValues = leafReader.getSortedDocValues("sorted");
+    assertEquals("b", sortedDocValues.get(0).utf8ToString());
+    assertEquals(0, sortedDocValues.getOrd(0));
+    assertEquals("b", sortedDocValues.lookupOrd(0).utf8ToString());
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    assertEquals(3, sortedSetDocValues.getValueCount());
+    sortedSetDocValues.setDocument(0);
+    assertEquals(0L, sortedSetDocValues.nextOrd());
+    assertEquals(1L, sortedSetDocValues.nextOrd());
+    assertEquals(2L, sortedSetDocValues.nextOrd());
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+    assertEquals("c", sortedSetDocValues.lookupOrd(0L).utf8ToString());
+    assertEquals("d", sortedSetDocValues.lookupOrd(1L).utf8ToString());
+    assertEquals("f", sortedSetDocValues.lookupOrd(2L).utf8ToString());
+  }
+
+  public void testInvalidDocValuesUsage() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("30")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Can't add [BINARY] doc values field [field], because [NUMERIC] doc values field already exists", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new NumericDocValuesField("field", 30L));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [NUMERIC] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("a")));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("b")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [BINARY] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new SortedDocValuesField("field", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("field", new BytesRef("b")));
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [SORTED] doc values field [field]", e.getMessage());
+    }
+  }
+
+  public void testDocValuesDoNotAffectBoostPositionsOrOffset() throws Exception {
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    doc.add(new TextField("text", "quick brown fox", Field.Store.NO));
+    MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer, true, true);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+    TermsEnum tenum = leafReader.terms("text").iterator();
+
+    assertEquals("brown", tenum.next().utf8ToString());
+    PostingsEnum penum = tenum.postings(null, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(1, penum.nextPosition());
+    assertEquals(6, penum.startOffset());
+    assertEquals(11, penum.endOffset());
+
+    assertEquals("fox", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(2, penum.nextPosition());
+    assertEquals(12, penum.startOffset());
+    assertEquals(15, penum.endOffset());
+
+    assertEquals("quick", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(0, penum.nextPosition());
+    assertEquals(0, penum.startOffset());
+    assertEquals(5, penum.endOffset());
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("text");
+    assertEquals("quick brown fox", binaryDocValues.get(0).utf8ToString());
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/559432fc/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 57e25fe..3e6778a 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
@@ -35,10 +35,18 @@ import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.LegacyLongField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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;
@@ -52,6 +60,9 @@ import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiFields;
 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;
@@ -434,6 +445,129 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
     assertNull(reader.terms("not-in-index"));
   }
 
+  public void testDocValuesMemoryIndexVsNormalIndex() throws Exception {
+    Document doc = new Document();
+    long randomLong = random().nextLong();
+    doc.add(new NumericDocValuesField("numeric", randomLong));
+    if (random().nextBoolean()) {
+      doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+    }
+    int numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomLong = random().nextLong();
+      doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      }
+      if (random().nextBoolean()) {
+        doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+      }
+    }
+    BytesRef randomTerm = new BytesRef(randomTerm());
+    doc.add(new BinaryDocValuesField("binary", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("binary", randomTerm, Field.Store.NO));
+    }
+    randomTerm = new BytesRef(randomTerm());
+    doc.add(new SortedDocValuesField("sorted", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("sorted", randomTerm, Field.Store.NO));
+    }
+    numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomTerm = new BytesRef(randomTerm());
+      doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      }
+      if (random().nextBoolean()) {
+        // randomily just add a normal string field
+        doc.add(new StringField("sorted_set", randomTerm, Field.Store.NO));
+      }
+    }
+
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+    MemoryIndex memoryIndex = MemoryIndex.fromDocument(doc, mockAnalyzer);
+    IndexReader indexReader = memoryIndex.createSearcher().getIndexReader();
+    LeafReader leafReader =  indexReader.leaves().get(0).reader();
+
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    NumericDocValues numericDocValues = leafReader.getNumericDocValues("numeric");
+    NumericDocValues controlNumericDocValues = controlLeafReader.getNumericDocValues("numeric");
+    assertEquals(controlNumericDocValues.get(0), numericDocValues.get(0));
+
+    SortedNumericDocValues sortedNumericDocValues = leafReader.getSortedNumericDocValues("sorted_numeric");
+    sortedNumericDocValues.setDocument(0);
+    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));
+    }
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    BinaryDocValues controlBinaryDocValues = controlLeafReader.getBinaryDocValues("binary");
+    assertEquals(controlBinaryDocValues.get(0), binaryDocValues.get(0));
+
+    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(controlSortedDocValues.lookupOrd(0), sortedDocValues.lookupOrd(0));
+
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    sortedSetDocValues.setDocument(0);
+    SortedSetDocValues controlSortedSetDocValues = controlLeafReader.getSortedSetDocValues("sorted_set");
+    controlSortedSetDocValues.setDocument(0);
+    assertEquals(controlSortedSetDocValues.getValueCount(), sortedSetDocValues.getValueCount());
+    for (long controlOrd = controlSortedSetDocValues.nextOrd(); controlOrd != SortedSetDocValues.NO_MORE_ORDS;
+         controlOrd = controlSortedSetDocValues.nextOrd()) {
+      assertEquals(controlOrd, sortedSetDocValues.nextOrd());
+      assertEquals(controlSortedSetDocValues.lookupOrd(controlOrd), sortedSetDocValues.lookupOrd(controlOrd));
+    }
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+
+    indexReader.close();
+    controlIndexReader.close();
+    dir.close();
+  }
+
+  public void testNormsWithDocValues() throws Exception {
+    MemoryIndex mi = new MemoryIndex(true, true);
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+
+    mi.addField(new BinaryDocValuesField("text", new BytesRef("quick brown fox")), mockAnalyzer, 5f);
+    mi.addField(new TextField("text", "quick brown fox", Field.Store.NO), mockAnalyzer, 5f);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    Field field = new TextField("text", "quick brown fox", Field.Store.NO);
+    field.setBoost(5f);
+    doc.add(field);
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    assertEquals(controlLeafReader.getNormValues("text").get(0), leafReader.getNormValues("text").get(0));
+
+    controlIndexReader.close();
+    dir.close();
+  }
+
   public void testDuellMemIndex() throws IOException {
     LineFileDocs lineFileDocs = new LineFileDocs(random());
     int numDocs = atLeast(10);


[3/3] lucene-solr git commit: LUCENE-7091: Added doc values support to memory index

Posted by mv...@apache.org.
LUCENE-7091: Added doc values support to memory index


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/bd0803fd
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/bd0803fd
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/bd0803fd

Branch: refs/heads/branch_6_0
Commit: bd0803fd41c68297c54201529c2c14ad50cda48e
Parents: 118b896
Author: Martijn van Groningen <ma...@gmail.com>
Authored: Wed Mar 9 17:38:23 2016 +0100
Committer: Martijn van Groningen <mv...@apache.org>
Committed: Tue Mar 15 10:01:25 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../apache/lucene/index/memory/MemoryIndex.java | 465 +++++++++++++++----
 .../lucene/index/memory/TestMemoryIndex.java    | 133 ++++++
 .../memory/TestMemoryIndexAgainstRAMDir.java    | 134 ++++++
 4 files changed, 636 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd0803fd/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a61d304..3ba94ae 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -178,6 +178,9 @@ Other
 * LUCENE-7087: Let MemoryIndex#fromDocument(...) accept 'Iterable<? extends IndexableField>'
   as document instead of 'Document'. (Martijn van Groningen)
 
+* LUCENE-7091: Add doc values support to MemoryIndex
+  (Martijn van Groningen, David Smiley)
+
 ======================= Lucene 5.5.0 =======================
 
 New Features

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd0803fd/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 9e01182..40159aa 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
@@ -17,12 +17,15 @@
 package org.apache.lucene.index.memory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.Locale;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -255,7 +258,7 @@ public class MemoryIndex {
       throw new IllegalArgumentException("analyzer must not be null");
     
     TokenStream stream = analyzer.tokenStream(fieldName, text);
-    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName));
+    addField(fieldName, stream, 1.0f, analyzer.getPositionIncrementGap(fieldName), analyzer.getOffsetGap(fieldName), DocValuesType.NONE, null);
   }
 
   /**
@@ -351,7 +354,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @throws IllegalArgumentException if the field is a DocValues or Point field, as these
@@ -362,7 +367,9 @@ public class MemoryIndex {
   }
 
   /**
-   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer
+   * Adds a lucene {@link IndexableField} to the MemoryIndex using the provided analyzer.
+   * Also stores doc values based on {@link IndexableFieldType#docValuesType()} if set.
+   *
    * @param field the field to add
    * @param analyzer the analyzer to use for term analysis
    * @param boost a field boost
@@ -370,17 +377,42 @@ public class MemoryIndex {
    *                                  structures are not supported by MemoryIndex
    */
   public void addField(IndexableField field, Analyzer analyzer, float boost) {
-    if (field.fieldType().docValuesType() != DocValuesType.NONE)
-      throw new IllegalArgumentException("MemoryIndex does not support DocValues fields");
-    if (field.fieldType().pointDimensionCount() != 0)
+    if (field.fieldType().pointDimensionCount() != 0) {
       throw new IllegalArgumentException("MemoryIndex does not support Points");
-    if (analyzer == null) {
-      addField(field.name(), field.tokenStream(null, null), boost);
     }
-    else {
-      addField(field.name(), field.tokenStream(analyzer, null), boost,
-          analyzer.getPositionIncrementGap(field.name()), analyzer.getOffsetGap(field.name()));
+
+    int offsetGap;
+    TokenStream tokenStream;
+    int positionIncrementGap;
+    if (analyzer != null) {
+      offsetGap = analyzer.getOffsetGap(field.name());
+      tokenStream = field.tokenStream(analyzer, null);
+      positionIncrementGap = analyzer.getPositionIncrementGap(field.name());
+    } else {
+      offsetGap = 1;
+      tokenStream = field.tokenStream(null, null);
+      positionIncrementGap = 0;
+    }
+
+    DocValuesType docValuesType = field.fieldType().docValuesType();
+    Object docValuesValue;
+    switch (docValuesType) {
+      case NONE:
+        docValuesValue = null;
+        break;
+      case BINARY:
+      case SORTED:
+      case SORTED_SET:
+        docValuesValue = field.binaryValue();
+        break;
+      case NUMERIC:
+      case SORTED_NUMERIC:
+        docValuesValue = field.numericValue();
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
     }
+    addField(field.name(), tokenStream, boost, positionIncrementGap, offsetGap, docValuesType, docValuesValue);
   }
   
   /**
@@ -448,66 +480,126 @@ public class MemoryIndex {
    *            the offset gap if fields with the same name are added more than once
    * @see org.apache.lucene.document.Field#setBoost(float)
    */
-  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap,
-                       int offsetGap) {
-    try (TokenStream stream = tokenStream) {
-      if (frozen)
-        throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
-      if (fieldName == null)
-        throw new IllegalArgumentException("fieldName must not be null");
-      if (stream == null)
-        throw new IllegalArgumentException("token stream must not be null");
-      if (boost <= 0.0f)
-        throw new IllegalArgumentException("boost factor must be greater than 0.0");
-      int numTokens = 0;
-      int numOverlapTokens = 0;
-      int pos = -1;
-      final BytesRefHash terms;
-      final SliceByteStartArray sliceArray;
-      Info info;
-      long sumTotalTermFreq = 0;
-      int offset = 0;
-      FieldInfo fieldInfo;
-      if ((info = fields.get(fieldName)) != null) {
-        fieldInfo = info.fieldInfo;
-        numTokens = info.numTokens;
-        numOverlapTokens = info.numOverlapTokens;
-        pos = info.lastPosition + positionIncrementGap;
-        offset = info.lastOffset + offsetGap;
-        terms = info.terms;
-        boost *= info.boost;
-        sliceArray = info.sliceArray;
-        sumTotalTermFreq = info.sumTotalTermFreq;
-      } else {
-        fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, this.storePayloads,
-                                  this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
-                                  DocValuesType.NONE, -1, Collections.emptyMap(), 0, 0);
-        sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
-        terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);
-      }
+  public void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    addField(fieldName, tokenStream, boost, positionIncrementGap, offsetGap, DocValuesType.NONE, null);
+  }
+
+  private void addField(String fieldName, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap,
+                        DocValuesType docValuesType, Object docValuesValue) {
+
+    if (frozen) {
+      throw new IllegalArgumentException("Cannot call addField() when MemoryIndex is frozen");
+    }
+    if (fieldName == null) {
+      throw new IllegalArgumentException("fieldName must not be null");
+    }
+    if (boost <= 0.0f) {
+      throw new IllegalArgumentException("boost factor must be greater than 0.0");
+    }
+
+    Info info = fields.get(fieldName);
+    if (info == null) {
+      IndexOptions indexOptions = storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+      FieldInfo fieldInfo = new FieldInfo(fieldName, fields.size(), true, false, storePayloads, indexOptions, docValuesType, -1, Collections.emptyMap(), 0, 0);
+      fields.put(fieldName, info = new Info(fieldInfo, byteBlockPool));
+    }
+
+    if (docValuesType != DocValuesType.NONE) {
+      storeDocValues(info, docValuesType, docValuesValue);
+    }
+    if (tokenStream != null) {
+      storeTerms(info, tokenStream, boost, positionIncrementGap, offsetGap);
+    }
+  }
 
+  private void storeDocValues(Info info, DocValuesType docValuesType, Object docValuesValue) {
+    String fieldName = info.fieldInfo.name;
+    DocValuesType existingDocValuesType = info.fieldInfo.getDocValuesType();
+    if (existingDocValuesType == DocValuesType.NONE) {
+      // first time we add doc values for this field:
+      info.fieldInfo = new FieldInfo(
+          info.fieldInfo.name, info.fieldInfo.number, info.fieldInfo.hasVectors(), info.fieldInfo.hasPayloads(),
+          info.fieldInfo.hasPayloads(), info.fieldInfo.getIndexOptions(), docValuesType, -1, info.fieldInfo.attributes(),
+          info.fieldInfo.getPointDimensionCount(), info.fieldInfo.getPointNumBytes()
+      );
+    } else if (existingDocValuesType != docValuesType) {
+      throw new IllegalArgumentException("Can't add [" + docValuesType + "] doc values field [" + fieldName + "], because [" + existingDocValuesType + "] doc values field already exists");
+    }
+    switch (docValuesType) {
+      case NUMERIC:
+        if (info.numericProducer.dvLongValues != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.numericProducer.dvLongValues = new long[]{(long) docValuesValue};
+        info.numericProducer.count++;
+        break;
+      case SORTED_NUMERIC:
+        if (info.numericProducer.dvLongValues == null) {
+          info.numericProducer.dvLongValues = new long[4];
+        }
+        info.numericProducer.dvLongValues = ArrayUtil.grow(info.numericProducer.dvLongValues, info.numericProducer.count + 1);
+        info.numericProducer.dvLongValues[info.numericProducer.count++] = (long) docValuesValue;
+        break;
+      case BINARY:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED:
+        if (info.binaryProducer.dvBytesValuesSet != null) {
+          throw new IllegalArgumentException("Only one value per field allowed for [" + docValuesType + "] doc values field [" + fieldName + "]");
+        }
+        info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      case SORTED_SET:
+        if (info.binaryProducer.dvBytesValuesSet == null) {
+          info.binaryProducer.dvBytesValuesSet = new BytesRefHash(byteBlockPool);
+        }
+        info.binaryProducer.dvBytesValuesSet.add((BytesRef) docValuesValue);
+        break;
+      default:
+        throw new UnsupportedOperationException("unknown doc values type [" + docValuesType + "]");
+    }
+  }
+
+  private void storeTerms(Info info, TokenStream tokenStream, float boost, int positionIncrementGap, int offsetGap) {
+    int pos = -1;
+    int offset = 0;
+    if (info.numTokens == 0) {
+      info.boost = boost;
+    } else if (info.numTokens > 0) {
+      pos = info.lastPosition + positionIncrementGap;
+      offset = info.lastOffset + offsetGap;
+      info.boost *= boost;
+    }
+
+    try (TokenStream stream = tokenStream) {
       TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
       PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
       OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
       PayloadAttribute payloadAtt = storePayloads ? stream.addAttribute(PayloadAttribute.class) : null;
       stream.reset();
-      
+
       while (stream.incrementToken()) {
 //        if (DEBUG) System.err.println("token='" + term + "'");
-        numTokens++;
+        info.numTokens++;
         final int posIncr = posIncrAttribute.getPositionIncrement();
-        if (posIncr == 0)
-          numOverlapTokens++;
+        if (posIncr == 0) {
+          info.numOverlapTokens++;
+        }
         pos += posIncr;
-        int ord = terms.add(termAtt.getBytesRef());
+        int ord = info.terms.add(termAtt.getBytesRef());
         if (ord < 0) {
           ord = (-ord) - 1;
-          postingsWriter.reset(sliceArray.end[ord]);
+          postingsWriter.reset(info.sliceArray.end[ord]);
         } else {
-          sliceArray.start[ord] = postingsWriter.startNewSlice();
+          info.sliceArray.start[ord] = postingsWriter.startNewSlice();
         }
-        sliceArray.freq[ord]++;
-        sumTotalTermFreq++;
+        info.sliceArray.freq[ord]++;
+        info.sumTotalTermFreq++;
         postingsWriter.writeInt(pos);
         if (storeOffsets) {
           postingsWriter.writeInt(offsetAtt.startOffset() + offset);
@@ -523,13 +615,12 @@ public class MemoryIndex {
           }
           postingsWriter.writeInt(pIndex);
         }
-        sliceArray.end[ord] = postingsWriter.getCurrentOffset();
+        info.sliceArray.end[ord] = postingsWriter.getCurrentOffset();
       }
       stream.end();
-
-      // ensure infos.numTokens > 0 invariant; needed for correct operation of terms()
-      if (numTokens > 0) {
-        fields.put(fieldName, new Info(fieldInfo, terms, sliceArray, numTokens, numOverlapTokens, boost, pos, offsetAtt.endOffset() + offset, sumTotalTermFreq));
+      if (info.numTokens > 0) {
+        info.lastPosition = pos;
+        info.lastOffset = offsetAtt.endOffset() + offset;
       }
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -573,8 +664,7 @@ public class MemoryIndex {
   public void freeze() {
     this.frozen = true;
     for (Info info : fields.values()) {
-      info.sortTerms();
-      info.getNormDocValues();//lazily computed
+      info.freeze();
     }
   }
   
@@ -702,7 +792,7 @@ public class MemoryIndex {
    */
   private final class Info {
 
-    private final FieldInfo fieldInfo;
+    private FieldInfo fieldInfo;
 
     /** The norms for this field; computed on demand. */
     private transient NumericDocValues norms;
@@ -711,40 +801,48 @@ public class MemoryIndex {
      * Term strings and their positions for this field: Map &lt;String
      * termText, ArrayIntList positions&gt;
      */
-    private final BytesRefHash terms; // note unfortunate variable name class with Terms type
+    private BytesRefHash terms; // note unfortunate variable name class with Terms type
     
-    private final SliceByteStartArray sliceArray;
+    private SliceByteStartArray sliceArray;
 
     /** Terms sorted ascending by term text; computed on demand */
     private transient int[] sortedTerms;
     
     /** Number of added tokens for this field */
-    private final int numTokens;
+    private int numTokens;
     
     /** Number of overlapping tokens for this field */
-    private final int numOverlapTokens;
+    private int numOverlapTokens;
     
     /** Boost factor for hits for this field */
-    private final float boost;
+    private float boost;
 
-    private final long sumTotalTermFreq;
+    private long sumTotalTermFreq;
 
     /** the last position encountered in this field for multi field support*/
-    private final int lastPosition;
+    private int lastPosition;
 
     /** the last offset encountered in this field for multi field support*/
-    private final int lastOffset;
+    private int lastOffset;
 
-    public Info(FieldInfo fieldInfo, BytesRefHash terms, SliceByteStartArray sliceArray, int numTokens, int numOverlapTokens, float boost, int lastPosition, int lastOffset, long sumTotalTermFreq) {
+    private BinaryDocValuesProducer binaryProducer;
+
+    private NumericDocValuesProducer numericProducer;
+
+    private boolean preparedDocValues;
+
+    private Info(FieldInfo fieldInfo, ByteBlockPool byteBlockPool) {
       this.fieldInfo = fieldInfo;
-      this.terms = terms;
-      this.sliceArray = sliceArray; 
-      this.numTokens = numTokens;
-      this.numOverlapTokens = numOverlapTokens;
-      this.boost = boost;
-      this.sumTotalTermFreq = sumTotalTermFreq;
-      this.lastPosition = lastPosition;
-      this.lastOffset = lastOffset;
+      this.sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
+      this.terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);;
+      this.binaryProducer = new BinaryDocValuesProducer();
+      this.numericProducer = new NumericDocValuesProducer();
+    }
+
+    void freeze() {
+      sortTerms();
+      prepareDocValues();
+      getNormDocValues();
     }
 
     /**
@@ -755,13 +853,26 @@ public class MemoryIndex {
      * (which would be an alternative and somewhat more elegant approach,
      * apart from more sophisticated Tries / prefix trees).
      */
-    public void sortTerms() {
+    void sortTerms() {
       if (sortedTerms == null) {
         sortedTerms = terms.sort();
       }
     }
 
-    public NumericDocValues getNormDocValues() {
+    void prepareDocValues() {
+      if (preparedDocValues == false) {
+        DocValuesType dvType = fieldInfo.getDocValuesType();
+        if (dvType == DocValuesType.NUMERIC || dvType == DocValuesType.SORTED_NUMERIC) {
+          numericProducer.prepareForUsage();
+        }
+        if (dvType == DocValuesType.BINARY || dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
+          binaryProducer.prepareForUsage();
+        }
+        preparedDocValues = true;
+      }
+    }
+
+    NumericDocValues getNormDocValues() {
       if (norms == null) {
         FieldInvertState invertState = new FieldInvertState(fieldInfo.name, fieldInfo.number,
             numTokens, numOverlapTokens, 0, boost);
@@ -786,7 +897,81 @@ public class MemoryIndex {
   ///////////////////////////////////////////////////////////////////////////////
   // Nested classes:
   ///////////////////////////////////////////////////////////////////////////////
-    
+
+  private static final class BinaryDocValuesProducer {
+
+    BytesRefHash dvBytesValuesSet;
+    final SortedDocValues sortedDocValues;
+    final BytesRef spare = new BytesRef();
+
+    int[] bytesIds;
+
+    private BinaryDocValuesProducer() {
+      sortedDocValues = new SortedDocValues() {
+        @Override
+        public int getOrd(int docID) {
+          return 0;
+        }
+
+        @Override
+        public BytesRef lookupOrd(int ord) {
+          return getValue(ord);
+        }
+
+        @Override
+        public int getValueCount() {
+          return 1;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      bytesIds = dvBytesValuesSet.sort();
+    }
+
+    private BytesRef getValue(int index) {
+      return dvBytesValuesSet.get(bytesIds[index], spare);
+    }
+
+  }
+
+  private static final class NumericDocValuesProducer {
+
+    long[] dvLongValues;
+    int count;
+
+    final NumericDocValues numericDocValues;
+    final SortedNumericDocValues sortedNumericDocValues;
+
+    private NumericDocValuesProducer() {
+      this.numericDocValues = new NumericDocValues() {
+        @Override
+        public long get(int docID) {
+          return dvLongValues[0];
+        }
+      };
+      this.sortedNumericDocValues = new SortedNumericDocValues() {
+        @Override
+        public void setDocument(int doc) {
+        }
+
+        @Override
+        public long valueAt(int index) {
+          return dvLongValues[index];
+        }
+
+        @Override
+        public int count() {
+          return count;
+        }
+      };
+    }
+
+    private void prepareForUsage() {
+      Arrays.sort(dvLongValues, 0, count);
+    }
+  }
+
   /**
    * Search support for Lucene framework integration; implements all methods
    * required by the Lucene IndexReader contracts.
@@ -795,6 +980,9 @@ public class MemoryIndex {
     
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
+      for (Info info : fields.values()) {
+        info.prepareDocValues();
+      }
     }
 
     @Override
@@ -807,8 +995,18 @@ public class MemoryIndex {
       removeCoreClosedListenerAsReaderClosedListener(this, listener);
     }
 
-    private Info getInfo(String fieldName) {
-      return fields.get(fieldName);
+    private Info getInfoForExpectedDocValuesType(String fieldName, DocValuesType expectedType) {
+      if (expectedType == DocValuesType.NONE) {
+        return null;
+      }
+      Info info = fields.get(fieldName);
+      if (info == null) {
+        return null;
+      }
+      if (info.fieldInfo.getDocValuesType() != expectedType) {
+        return null;
+      }
+      return info;
     }
 
     @Override
@@ -828,32 +1026,87 @@ public class MemoryIndex {
 
     @Override
     public NumericDocValues getNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.NUMERIC);
+      if (info != null) {
+        return info.numericProducer.numericDocValues;
+      } else {
+        return null;
+      }
     }
 
     @Override
     public BinaryDocValues getBinaryDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.BINARY);
     }
 
     @Override
     public SortedDocValues getSortedDocValues(String field) {
-      return null;
+      return getSortedDocValues(field, DocValuesType.SORTED);
+    }
+
+    private SortedDocValues getSortedDocValues(String field, DocValuesType docValuesType) {
+      Info info = getInfoForExpectedDocValuesType(field, docValuesType);
+      if (info != null) {
+        return info.binaryProducer.sortedDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedNumericDocValues getSortedNumericDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_NUMERIC);
+      if (info != null) {
+        return info.numericProducer.sortedNumericDocValues;
+      } else {
+        return null;
+      }
     }
     
     @Override
     public SortedSetDocValues getSortedSetDocValues(String field) {
-      return null;
+      Info info = getInfoForExpectedDocValuesType(field, DocValuesType.SORTED_SET);
+      if (info != null) {
+        return new SortedSetDocValues() {
+
+          int index = 0;
+
+          @Override
+          public long nextOrd() {
+            if (index >= info.binaryProducer.dvBytesValuesSet.size()) {
+              return NO_MORE_ORDS;
+            }
+            return index++;
+          }
+
+          @Override
+          public void setDocument(int docID) {
+            index = 0;
+          }
+
+          @Override
+          public BytesRef lookupOrd(long ord) {
+            return info.binaryProducer.getValue((int) ord);
+          }
+
+          @Override
+          public long getValueCount() {
+            return info.binaryProducer.dvBytesValuesSet.size();
+          }
+        };
+      } else {
+        return null;
+      }
     }
 
     @Override
     public Bits getDocsWithField(String field) throws IOException {
-      return null;
+      Info info = fields.get(field);
+      if (info != null && info.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
+        return new Bits.MatchAllBits(1);
+      } else {
+        return null;
+      }
     }
 
     @Override
@@ -866,7 +1119,25 @@ public class MemoryIndex {
       // no-op
     }
 
+    @Override
+    public Fields fields() {
+      Map<String, Info> filteredFields = fields.entrySet().stream()
+          .filter(entry ->  entry.getValue().numTokens > 0)
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
+              (u,v) -> { throw new IllegalStateException(String.format(Locale.ROOT, "Duplicate key %s", u));},
+              TreeMap::new
+          ));
+      return new MemoryFields(filteredFields );
+    }
+
     private class MemoryFields extends Fields {
+
+      private final Map<String, Info> fields;
+
+      public MemoryFields(Map<String, Info> fields) {
+        this.fields = fields;
+      }
+
       @Override
       public Iterator<String> iterator() {
         return fields.keySet().iterator();
@@ -875,8 +1146,9 @@ public class MemoryIndex {
       @Override
       public Terms terms(final String field) {
         final Info info = fields.get(field);
-        if (info == null)
+        if (info == null) {
           return null;
+        }
 
         return new Terms() {
           @Override
@@ -932,11 +1204,6 @@ public class MemoryIndex {
         return fields.size();
       }
     }
-  
-    @Override
-    public Fields fields() {
-      return new MemoryFields();
-    }
 
     private class MemoryTermsEnum extends TermsEnum {
       private final Info info;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd0803fd/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 b150ea3..7282e0e 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
@@ -21,13 +21,25 @@ import java.io.IOException;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockPayloadAnalyzer;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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.FieldInvertState;
 import org.apache.lucene.index.LeafReader;
+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.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -35,6 +47,7 @@ import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.similarities.BM25Similarity;
 import org.apache.lucene.search.similarities.ClassicSimilarity;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Before;
@@ -176,5 +189,125 @@ public class TestMemoryIndex extends LuceneTestCase {
 
   }
 
+  public void testDocValues() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("numeric", 29L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 33L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 32L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 31L));
+    doc.add(new SortedNumericDocValuesField("sorted_numeric", 30L));
+    doc.add(new BinaryDocValuesField("binary", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("sorted", new BytesRef("b")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("f")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("d")));
+    doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef("c")));
+
+    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));
+    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));
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    assertEquals("a", binaryDocValues.get(0).utf8ToString());
+    SortedDocValues sortedDocValues = leafReader.getSortedDocValues("sorted");
+    assertEquals("b", sortedDocValues.get(0).utf8ToString());
+    assertEquals(0, sortedDocValues.getOrd(0));
+    assertEquals("b", sortedDocValues.lookupOrd(0).utf8ToString());
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    assertEquals(3, sortedSetDocValues.getValueCount());
+    sortedSetDocValues.setDocument(0);
+    assertEquals(0L, sortedSetDocValues.nextOrd());
+    assertEquals(1L, sortedSetDocValues.nextOrd());
+    assertEquals(2L, sortedSetDocValues.nextOrd());
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+    assertEquals("c", sortedSetDocValues.lookupOrd(0L).utf8ToString());
+    assertEquals("d", sortedSetDocValues.lookupOrd(1L).utf8ToString());
+    assertEquals("f", sortedSetDocValues.lookupOrd(2L).utf8ToString());
+  }
+
+  public void testInvalidDocValuesUsage() throws Exception {
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("30")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Can't add [BINARY] doc values field [field], because [NUMERIC] doc values field already exists", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new NumericDocValuesField("field", 29L));
+    doc.add(new NumericDocValuesField("field", 30L));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [NUMERIC] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("a")));
+    doc.add(new BinaryDocValuesField("field", new BytesRef("b")));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [BINARY] doc values field [field]", e.getMessage());
+    }
+
+    doc = new Document();
+    doc.add(new SortedDocValuesField("field", new BytesRef("a")));
+    doc.add(new SortedDocValuesField("field", new BytesRef("b")));
+    doc.add(new TextField("field", "a b", Field.Store.NO));
+    try {
+      MemoryIndex.fromDocument(doc, analyzer);
+    } catch (IllegalArgumentException e) {
+      assertEquals("Only one value per field allowed for [SORTED] doc values field [field]", e.getMessage());
+    }
+  }
+
+  public void testDocValuesDoNotAffectBoostPositionsOrOffset() throws Exception {
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    doc.add(new TextField("text", "quick brown fox", Field.Store.NO));
+    MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer, true, true);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+    TermsEnum tenum = leafReader.terms("text").iterator();
+
+    assertEquals("brown", tenum.next().utf8ToString());
+    PostingsEnum penum = tenum.postings(null, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(1, penum.nextPosition());
+    assertEquals(6, penum.startOffset());
+    assertEquals(11, penum.endOffset());
+
+    assertEquals("fox", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(2, penum.nextPosition());
+    assertEquals(12, penum.startOffset());
+    assertEquals(15, penum.endOffset());
+
+    assertEquals("quick", tenum.next().utf8ToString());
+    penum = tenum.postings(penum, PostingsEnum.OFFSETS);
+    assertEquals(0, penum.nextDoc());
+    assertEquals(1, penum.freq());
+    assertEquals(0, penum.nextPosition());
+    assertEquals(0, penum.startOffset());
+    assertEquals(5, penum.endOffset());
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("text");
+    assertEquals("quick brown fox", binaryDocValues.get(0).utf8ToString());
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd0803fd/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 57e25fe..3e6778a 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
@@ -35,10 +35,18 @@ import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.LegacyLongField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+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;
@@ -52,6 +60,9 @@ import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiFields;
 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;
@@ -434,6 +445,129 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
     assertNull(reader.terms("not-in-index"));
   }
 
+  public void testDocValuesMemoryIndexVsNormalIndex() throws Exception {
+    Document doc = new Document();
+    long randomLong = random().nextLong();
+    doc.add(new NumericDocValuesField("numeric", randomLong));
+    if (random().nextBoolean()) {
+      doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+    }
+    int numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomLong = random().nextLong();
+      doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
+      }
+      if (random().nextBoolean()) {
+        doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
+      }
+    }
+    BytesRef randomTerm = new BytesRef(randomTerm());
+    doc.add(new BinaryDocValuesField("binary", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("binary", randomTerm, Field.Store.NO));
+    }
+    randomTerm = new BytesRef(randomTerm());
+    doc.add(new SortedDocValuesField("sorted", randomTerm));
+    if (random().nextBoolean()) {
+      doc.add(new StringField("sorted", randomTerm, Field.Store.NO));
+    }
+    numValues = atLeast(5);
+    for (int i = 0; i < numValues; i++) {
+      randomTerm = new BytesRef(randomTerm());
+      doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      if (random().nextBoolean()) {
+        // randomly duplicate field/value
+        doc.add(new SortedSetDocValuesField("sorted_set", randomTerm));
+      }
+      if (random().nextBoolean()) {
+        // randomily just add a normal string field
+        doc.add(new StringField("sorted_set", randomTerm, Field.Store.NO));
+      }
+    }
+
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+    MemoryIndex memoryIndex = MemoryIndex.fromDocument(doc, mockAnalyzer);
+    IndexReader indexReader = memoryIndex.createSearcher().getIndexReader();
+    LeafReader leafReader =  indexReader.leaves().get(0).reader();
+
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    NumericDocValues numericDocValues = leafReader.getNumericDocValues("numeric");
+    NumericDocValues controlNumericDocValues = controlLeafReader.getNumericDocValues("numeric");
+    assertEquals(controlNumericDocValues.get(0), numericDocValues.get(0));
+
+    SortedNumericDocValues sortedNumericDocValues = leafReader.getSortedNumericDocValues("sorted_numeric");
+    sortedNumericDocValues.setDocument(0);
+    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));
+    }
+
+    BinaryDocValues binaryDocValues = leafReader.getBinaryDocValues("binary");
+    BinaryDocValues controlBinaryDocValues = controlLeafReader.getBinaryDocValues("binary");
+    assertEquals(controlBinaryDocValues.get(0), binaryDocValues.get(0));
+
+    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(controlSortedDocValues.lookupOrd(0), sortedDocValues.lookupOrd(0));
+
+    SortedSetDocValues sortedSetDocValues = leafReader.getSortedSetDocValues("sorted_set");
+    sortedSetDocValues.setDocument(0);
+    SortedSetDocValues controlSortedSetDocValues = controlLeafReader.getSortedSetDocValues("sorted_set");
+    controlSortedSetDocValues.setDocument(0);
+    assertEquals(controlSortedSetDocValues.getValueCount(), sortedSetDocValues.getValueCount());
+    for (long controlOrd = controlSortedSetDocValues.nextOrd(); controlOrd != SortedSetDocValues.NO_MORE_ORDS;
+         controlOrd = controlSortedSetDocValues.nextOrd()) {
+      assertEquals(controlOrd, sortedSetDocValues.nextOrd());
+      assertEquals(controlSortedSetDocValues.lookupOrd(controlOrd), sortedSetDocValues.lookupOrd(controlOrd));
+    }
+    assertEquals(SortedSetDocValues.NO_MORE_ORDS, sortedSetDocValues.nextOrd());
+
+    indexReader.close();
+    controlIndexReader.close();
+    dir.close();
+  }
+
+  public void testNormsWithDocValues() throws Exception {
+    MemoryIndex mi = new MemoryIndex(true, true);
+    MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
+
+    mi.addField(new BinaryDocValuesField("text", new BytesRef("quick brown fox")), mockAnalyzer, 5f);
+    mi.addField(new TextField("text", "quick brown fox", Field.Store.NO), mockAnalyzer, 5f);
+    LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
+
+    Document doc = new Document();
+    doc.add(new BinaryDocValuesField("text", new BytesRef("quick brown fox")));
+    Field field = new TextField("text", "quick brown fox", Field.Store.NO);
+    field.setBoost(5f);
+    doc.add(field);
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(random(), mockAnalyzer));
+    writer.addDocument(doc);
+    writer.close();
+
+    IndexReader controlIndexReader = DirectoryReader.open(dir);
+    LeafReader controlLeafReader =  controlIndexReader.leaves().get(0).reader();
+
+    assertEquals(controlLeafReader.getNormValues("text").get(0), leafReader.getNormValues("text").get(0));
+
+    controlIndexReader.close();
+    dir.close();
+  }
+
   public void testDuellMemIndex() throws IOException {
     LineFileDocs lineFileDocs = new LineFileDocs(random());
     int numDocs = atLeast(10);