You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2018/08/20 17:41:08 UTC

lucene-solr:branch_7x: SOLR-12625: Combine SolrDocumentFetcher and RetrieveFieldsOptimizer

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 569d77103 -> 82c64af84


SOLR-12625: Combine SolrDocumentFetcher and RetrieveFieldsOptimizer

(cherry picked from commit 66d500b5a59e1aefe9170e8c5cb70a9e0b0f1033)


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

Branch: refs/heads/branch_7x
Commit: 82c64af84b903df40e457ed6e338b3abf43a7534
Parents: 569d771
Author: Erick Erickson <Er...@gmail.com>
Authored: Mon Aug 20 10:37:32 2018 -0700
Committer: Erick Erickson <Er...@gmail.com>
Committed: Mon Aug 20 10:38:36 2018 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../handler/component/TermVectorComponent.java  |  41 +-
 .../org/apache/solr/response/DocsStreamer.java  |  26 +-
 .../solr/response/RetrieveFieldsOptimizer.java  | 130 ----
 .../apache/solr/search/SolrDocumentFetcher.java | 198 +++++-
 .../apache/solr/search/SolrReturnFields.java    |  31 +
 .../response/TestRetrieveFieldsOptimizer.java   | 642 ++++++++++++++++++-
 solr/solr-ref-guide/src/docvalues.adoc          |  20 +-
 8 files changed, 872 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5471825..87131bc 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -206,6 +206,8 @@ Bug Fixes
 
 * SOLR-12679: MiniSolrCloudCluster internal jetty list should never have duplicates (shalin)
 
+* SOLR-12598: Do not fetch non-stored fields (Nikolay Khitrin, Erick Erickson)
+
 Optimizations
 ----------------------
 
@@ -282,6 +284,8 @@ Other Changes
   not require node restarts and CdcrWithNodesRestartsTest which does. The tests themselves are made faster and more
   resilient to spurious failures. (Varun Thacker, Amrit Sarkar via shalin)
 
+* SOLR-12625: Combine SolrDocumentFetcher and RetrieveFieldsOptimizer (Erick Erickson)
+
 ==================  7.4.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
index 784dc81..8d7617e 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
@@ -44,8 +43,6 @@ import org.apache.solr.common.params.TermVectorParams;
 import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.response.DocsStreamer;
-import org.apache.solr.response.RetrieveFieldsOptimizer;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocList;
@@ -267,41 +264,27 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
     //field
     SolrDocumentFetcher docFetcher = searcher.getDocFetcher();
     SolrReturnFields srf = new SolrReturnFields(uniqFieldName, rb.req);
-    RetrieveFieldsOptimizer retrieveFieldsOptimizer = RetrieveFieldsOptimizer.create(docFetcher, srf);
 
     while (iter.hasNext()) {
       Integer docId = iter.next();
       NamedList<Object> docNL = new NamedList<>();
 
       if (keyField != null) {
-        SolrDocument sdoc = null;
-        try {
-          if (retrieveFieldsOptimizer.returnStoredFields()) {
-            Document doc = docFetcher.doc(docId, retrieveFieldsOptimizer.getStoredFields());
-            // make sure to use the schema from the searcher and not the request (cross-core)
-            sdoc = DocsStreamer.convertLuceneDocToSolrDoc(doc, searcher.getSchema(), srf);
+        // guaranteed to be one and only one since this is uniqueKey!
+        SolrDocument solrDoc = docFetcher.solrDoc(docId, srf);
+
+        String uKey = null;
+        Object val = solrDoc.getFieldValue(uniqFieldName);
+        if (val != null) {
+          if (val instanceof StoredField) {
+            uKey = ((StoredField) val).stringValue();
           } else {
-            // no need to get stored fields of the document, see SOLR-5968
-            sdoc = new SolrDocument();
-          }
-
-          // decorate the document with non-stored docValues fields
-          if (retrieveFieldsOptimizer.returnDVFields()) {
-            docFetcher.decorateDocValueFields(sdoc, docId, retrieveFieldsOptimizer.getDvFields());
+            uKey = val.toString();
           }
-        } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document with docId " + docId, e);
         }
-        Object val = sdoc.getFieldValue(uniqFieldName);
-        String uniqVal = "";
-        if (val instanceof StoredField) {
-          uniqVal = ((StoredField) val).stringValue();
-        } else {
-          uniqVal = val.toString();
-        }
-
-        docNL.add("uniqueKey", uniqVal);
-        termVectors.add(uniqVal, docNL);
+        assert null != uKey;
+        docNL.add("uniqueKey", uKey);
+        termVectors.add(uKey, docNL);
       } else {
         // support for schemas w/o a unique key,
         termVectors.add("doc-" + docId, docNL);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
index ce2094b..67892cb 100644
--- a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
+++ b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java
@@ -66,7 +66,7 @@ public class DocsStreamer implements Iterator<SolrDocument> {
   private final DocTransformer transformer;
   private final DocIterator docIterator;
 
-  private final RetrieveFieldsOptimizer retrieveFieldsOptimizer;
+  private final SolrReturnFields solrReturnFields;
 
   private int idx = -1;
 
@@ -76,9 +76,8 @@ public class DocsStreamer implements Iterator<SolrDocument> {
     transformer = rctx.getReturnFields().getTransformer();
     docIterator = this.docs.iterator();
     docFetcher = rctx.getSearcher().getDocFetcher();
+    solrReturnFields = (SolrReturnFields)rctx.getReturnFields();
 
-    retrieveFieldsOptimizer = RetrieveFieldsOptimizer.create(docFetcher, rctx.getReturnFields());
-    retrieveFieldsOptimizer.optimize(docFetcher);
     if (transformer != null) transformer.setContext(rctx);
   }
 
@@ -93,26 +92,7 @@ public class DocsStreamer implements Iterator<SolrDocument> {
   public SolrDocument next() {
     int id = docIterator.nextDoc();
     idx++;
-    SolrDocument sdoc = null;
-
-    try {
-      if (retrieveFieldsOptimizer.returnStoredFields()) {
-        Document doc = docFetcher.doc(id, retrieveFieldsOptimizer.getStoredFields());
-        // make sure to use the schema from the searcher and not the request (cross-core)
-        sdoc = convertLuceneDocToSolrDoc(doc, rctx.getSearcher().getSchema(),
-                                         rctx.getReturnFields());
-      } else {
-        // no need to get stored fields of the document, see SOLR-5968
-        sdoc = new SolrDocument();
-      }
-
-      // decorate the document with non-stored docValues fields
-      if (retrieveFieldsOptimizer.returnDVFields()) {
-        docFetcher.decorateDocValueFields(sdoc, id, retrieveFieldsOptimizer.getDvFields());
-      }
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document with docId " + id, e);
-    }
+    SolrDocument sdoc = docFetcher.solrDoc(id, solrReturnFields);
 
     if (transformer != null) {
       boolean doScore = rctx.wantsScores();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/java/org/apache/solr/response/RetrieveFieldsOptimizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/RetrieveFieldsOptimizer.java b/solr/core/src/java/org/apache/solr/response/RetrieveFieldsOptimizer.java
deleted file mode 100644
index fcd47a7..0000000
--- a/solr/core/src/java/org/apache/solr/response/RetrieveFieldsOptimizer.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.solr.response;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.solr.search.ReturnFields;
-import org.apache.solr.search.SolrDocumentFetcher;
-import org.apache.solr.search.SolrReturnFields;
-
-public class RetrieveFieldsOptimizer {
-  // null means get all available stored fields
-  private final Set<String> storedFields;
-  // always non null
-  private final Set<String> dvFields;
-
-  RetrieveFieldsOptimizer(Set<String> storedFields, Set<String> dvFields) {
-    this.storedFields = storedFields;
-    this.dvFields = dvFields;
-  }
-
-  /**
-   * Sometimes we could fetch a field value from either the stored document or docValues.
-   * Such fields have both and are single-valued.
-   * If choosing docValues allows us to avoid accessing the stored document altogether
-   * for all fields to be returned then we do it,
-   * otherwise we prefer the stored value when we have a choice.
-   */
-  void optimize(SolrDocumentFetcher docFetcher) {
-    optimize(docFetcher.getDvsCanSubstituteStored());
-  }
-
-  void optimize(Set<String> dvsCanSubstituteStored) {
-    if (storedFields == null) return;
-    if (!dvsCanSubstituteStored.containsAll(storedFields)) return;
-    dvFields.addAll(storedFields);
-    storedFields.clear();
-  }
-
-  public boolean returnStoredFields() {
-    return !(storedFields != null && storedFields.isEmpty());
-  }
-
-  public boolean returnDVFields() {
-    return !dvFields.isEmpty();
-  }
-
-  public Set<String> getStoredFields() {
-    return storedFields;
-  }
-
-  public Set<String> getDvFields() {
-    return dvFields;
-  }
-
-  public static RetrieveFieldsOptimizer create(SolrDocumentFetcher docFetcher, ReturnFields returnFields) {
-    Set<String> storedFields = calcStoredFieldsForReturn(docFetcher, returnFields);
-    Set<String> dvFields = calcDocValueFieldsForReturn(docFetcher, returnFields);
-
-    return new RetrieveFieldsOptimizer(storedFields, dvFields);
-  }
-
-  private static Set<String> calcStoredFieldsForReturn(SolrDocumentFetcher docFetcher, ReturnFields returnFields) {
-    final Set<String> storedFields = new HashSet<>();
-    Set<String> fnames = returnFields.getLuceneFieldNames();
-    if (returnFields.wantsAllFields()) {
-      return null;
-    } else if (returnFields.hasPatternMatching()) {
-      for (String s : docFetcher.getAllStored()) {
-        if (returnFields.wantsField(s)) {
-          storedFields.add(s);
-        }
-      }
-    } else if (fnames != null) {
-      storedFields.addAll(fnames);
-    }
-    storedFields.remove(SolrReturnFields.SCORE);
-    return storedFields;
-  }
-
-  private static Set<String> calcDocValueFieldsForReturn(SolrDocumentFetcher docFetcher, ReturnFields returnFields) {
-    // always return not null
-    final Set<String> result = new HashSet<>();
-    if (returnFields.wantsAllFields()) {
-      result.addAll(docFetcher.getNonStoredDVs(true));
-      // check whether there are no additional fields
-      Set<String> fieldNames = returnFields.getLuceneFieldNames(true);
-      if (fieldNames != null) {
-        // add all requested fields that may be useDocValuesAsStored=false
-        for (String fl : fieldNames) {
-          if (docFetcher.getNonStoredDVs(false).contains(fl)) {
-            result.add(fl);
-          }
-        }
-      }
-    } else if (returnFields.hasPatternMatching()) {
-      for (String s : docFetcher.getNonStoredDVs(true)) {
-        if (returnFields.wantsField(s)) {
-          result.add(s);
-        }
-      }
-    } else {
-      Set<String> fnames = returnFields.getLuceneFieldNames();
-      if (fnames != null) {
-        result.addAll(fnames);
-        // here we get all non-stored dv fields because even if a user has set
-        // useDocValuesAsStored=false in schema, he may have requested a field
-        // explicitly using the fl parameter
-        result.retainAll(docFetcher.getNonStoredDVs(false));
-      }
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
index dbb8bd7..45c1cf2 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
@@ -30,7 +30,9 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.function.Supplier;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
@@ -53,8 +55,11 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.NumericUtils;
+import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentBase;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.SolrConfig;
+import org.apache.solr.response.DocsStreamer;
 import org.apache.solr.schema.BoolField;
 import org.apache.solr.schema.LatLonPointSpatialField;
 import org.apache.solr.schema.AbstractEnumField;
@@ -145,6 +150,8 @@ public class SolrDocumentFetcher {
     this.allStored = Collections.unmodifiableSet(allStoreds);
   }
 
+  // Does this field have both stored=true and docValues=true and is otherwise
+  // eligible for getting the field's value from DV?
   private boolean canSubstituteDvForStored(FieldInfo fieldInfo, SchemaField schemaField) {
     if (!schemaField.hasDocValues() || !schemaField.stored()) return false;
     if (schemaField.multiValued()) return false;
@@ -228,6 +235,43 @@ public class SolrDocumentFetcher {
     return d;
   }
 
+  /**
+   * This is an optimized version for populating a SolrDocument that:
+   *
+   * 1. fetches all fields from docValues if possible. If no decompression of the stored
+   * data is necessary, we can avoid a disk seek and decompression cycle.
+   * This step is only used if all requested fields are
+   * {code docValues=true stored=false multiValued=false}.
+   * This last restriction because multiValued docValues fields do not faithfully reflect
+   * the input order in all cases. the values are returned and no decompression is necessary.
+   *
+   * 2. if 1 is impossible, try to fetch all requested fields from the stored values. If
+   * the stored data has to be decompressed anyway, it's more efficient to
+   * just get all field values from the stored values. If we got all the requested fields, return.
+   *
+   * 3. add fields where docValues=true stored=false thus could not be fetched in step 2
+   *
+   * @param luceneDocId       The Lucene doc ID
+   * @param solrReturnFields  the structure holding the fields to be returned.
+   *                          The first time this method is called for a particular
+   *                          document list, it will be modified by adding a
+   *                          RetrieveFieldsOptimizer for use in future calls.
+   *
+   * @return The SolrDocument with values requested.
+   * <p>
+   * This method is designed to be as simple as possible to use, just call it. e.g.
+   * {code SolrDocument sdoc = docFetcher.solrDoc(id, solrReturnFields);}
+   * then process the resulting SolrDocument as usual. Subsequent calls with the same
+   * solrReturnFields will re-use the optimizer created the first time.
+   *
+   * NOTE: DO NOT re-use the same SolrReturnFields object if the fields requested change.
+   */
+
+  public SolrDocument solrDoc(int luceneDocId, SolrReturnFields solrReturnFields) {
+    Supplier<RetrieveFieldsOptimizer> rfoSupplier = () -> new RetrieveFieldsOptimizer(solrReturnFields);
+    return solrReturnFields.getFetchOptimizer(rfoSupplier).getSolrDoc(luceneDocId);
+  }
+
   /** {@link StoredFieldVisitor} which loads the specified fields eagerly (or all if null).
    * If {@link #enableLazyFieldLoading} then the rest get special lazy field entries.  Designated "large"
    * fields will always get a special field entry. */
@@ -352,6 +396,7 @@ public class SolrDocumentFetcher {
     public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
       return analyzer.tokenStream(name(), stringValue()); // or we could throw unsupported exception?
     }
+
     /** (for tests) */
     synchronized boolean hasBeenLoaded() {
       return cachedBytes != null;
@@ -373,6 +418,7 @@ public class SolrDocumentFetcher {
         BytesRef bytesRef = new BytesRef();
         searcher.getIndexReader().document(docId, new StoredFieldVisitor() {
           boolean done = false;
+
           @Override
           public Status needsField(FieldInfo fieldInfo) throws IOException {
             if (done) {
@@ -497,8 +543,7 @@ public class SolrDocumentFetcher {
             // normally never true but LatLonPointSpatialField uses SORTED_NUMERIC even when single valued
             else if (schemaField.multiValued() == false) {
               return value;
-            }
-            else {
+            } else {
               outValues.add(value);
             }
           }
@@ -529,7 +574,7 @@ public class SolrDocumentFetcher {
     if (schemaField.getType() instanceof LatLonPointSpatialField) {
       return LatLonPointSpatialField.decodeDocValueToString(value);
     }
-    
+
     if (schemaField.getType().getNumberType() == null) {
       log.warn("Couldn't decode docValues for field: [{}], schemaField: [{}], numberType is unknown",
           schemaField.getName(), schemaField);
@@ -592,4 +637,151 @@ public class SolrDocumentFetcher {
     return nonStoredDVsWithoutCopyTargets;
   }
 
+
+  /**
+   * Moved as a private class here, we consider it an impelmentation detail. It should not
+   * be exposed outside of this class.
+   * <p>
+   * This class is in charge of insuring that SolrDocuments can have their fields populated
+   * during a request in the most efficient way possible. See the comments at
+   * {@link #solrDoc(int docId, SolrReturnFields solrReturnFields)}
+   */
+
+  class RetrieveFieldsOptimizer {
+    // null means get all available stored fields
+    private final Set<String> storedFields;
+    // always non null
+    private final Set<String> dvFields;
+
+    private final SolrReturnFields solrReturnFields;
+
+    RetrieveFieldsOptimizer(SolrReturnFields solrReturnFields) {
+      this.storedFields = calcStoredFieldsForReturn(solrReturnFields);
+      this.dvFields = calcDocValueFieldsForReturn(solrReturnFields);
+      this.solrReturnFields = solrReturnFields;
+
+      if (storedFields != null && dvsCanSubstituteStored.containsAll(storedFields)) {
+        dvFields.addAll(storedFields);
+        storedFields.clear();
+      }
+    }
+
+    /**
+     * Sometimes we could fetch a field value from either the stored document or docValues.
+     * Such fields have both and are single-valued.
+     * If choosing docValues allows us to avoid accessing the stored document altogether
+     * for all fields to be returned then we do it,
+     * otherwise we prefer the stored value when we have a choice.
+     */
+    private boolean returnStoredFields() {
+      return !(storedFields != null && storedFields.isEmpty());
+    }
+
+    private boolean returnDVFields() {
+      return CollectionUtils.isNotEmpty(dvFields);
+    }
+
+    private Set<String> getStoredFields() {
+      return storedFields;
+    }
+
+    private Set<String> getDvFields() {
+      return dvFields;
+    }
+
+    //who uses all of these?
+    private ReturnFields getReturnFields() {
+      return solrReturnFields;
+    }
+
+    private Set<String> calcStoredFieldsForReturn(ReturnFields returnFields) {
+      final Set<String> storedFields = new HashSet<>();
+      Set<String> fnames = returnFields.getLuceneFieldNames();
+      if (returnFields.wantsAllFields()) {
+        return null;
+      } else if (returnFields.hasPatternMatching()) {
+        for (String s : getAllStored()) {
+          if (returnFields.wantsField(s)) {
+            storedFields.add(s);
+          }
+        }
+      } else if (fnames != null) {
+        storedFields.addAll(fnames);
+        storedFields.removeIf((String name) -> {
+          SchemaField schemaField = searcher.getSchema().getFieldOrNull(name);
+          if (schemaField == null) return false; // Get it from the stored fields if, for some reasonm, we can't get the schema.
+          if (schemaField.stored() && schemaField.multiValued()) return false; // must return multivalued fields from stored data if possible.
+          if (schemaField.stored() == false) return true; // if it's not stored, no choice but to return from DV.
+          return false;
+        });
+      }
+      storedFields.remove(SolrReturnFields.SCORE);
+      return storedFields;
+    }
+
+    private Set<String> calcDocValueFieldsForReturn(ReturnFields returnFields) {
+      // always return not null
+      final Set<String> result = new HashSet<>();
+      if (returnFields.wantsAllFields()) {
+        result.addAll(getNonStoredDVs(true));
+        // check whether there are no additional fields
+        Set<String> fieldNames = returnFields.getLuceneFieldNames(true);
+        if (fieldNames != null) {
+          // add all requested fields that may be useDocValuesAsStored=false
+          for (String fl : fieldNames) {
+            if (getNonStoredDVs(false).contains(fl)) {
+              result.add(fl);
+            }
+          }
+        }
+      } else if (returnFields.hasPatternMatching()) {
+        for (String s : getNonStoredDVs(true)) {
+          if (returnFields.wantsField(s)) {
+            result.add(s);
+          }
+        }
+      } else {
+        Set<String> fnames = returnFields.getLuceneFieldNames();
+        if (fnames != null) {
+          result.addAll(fnames);
+          // here we get all non-stored dv fields because even if a user has set
+          // useDocValuesAsStored=false in schema, he may have requested a field
+          // explicitly using the fl parameter
+          result.retainAll(getNonStoredDVs(false));
+        }
+      }
+      return result;
+    }
+
+    private SolrDocument getSolrDoc(int luceneDocId) {
+
+      SolrDocument sdoc = null;
+      try {
+        if (returnStoredFields()) {
+          Document doc = doc(luceneDocId, getStoredFields());
+          // make sure to use the schema from the searcher and not the request (cross-core)
+          sdoc = DocsStreamer.convertLuceneDocToSolrDoc(doc, searcher.getSchema(), getReturnFields());
+          if (returnDVFields() == false) {
+            solrReturnFields.setFieldSources(SolrReturnFields.FIELD_SOURCES.ALL_FROM_STORED);
+            return sdoc;
+          } else {
+            solrReturnFields.setFieldSources(SolrReturnFields.FIELD_SOURCES.MIXED_SOURCES);
+          }
+        } else {
+          // no need to get stored fields of the document, see SOLR-5968
+          sdoc = new SolrDocument();
+          solrReturnFields.setFieldSources(SolrReturnFields.FIELD_SOURCES.ALL_FROM_DV);
+        }
+
+        // decorate the document with non-stored docValues fields
+        if (returnDVFields()) {
+          decorateDocValueFields(sdoc, luceneDocId, getDvFields());
+        }
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document with docId " + luceneDocId, e);
+      }
+      return sdoc;
+    }
+  }
 }
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java b/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
index 2b1b303..d9e681b 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrReturnFields.java
@@ -33,6 +33,7 @@ import org.apache.solr.response.transform.RenameFieldTransformer;
 import org.apache.solr.response.transform.ScoreAugmenter;
 import org.apache.solr.response.transform.TransformerFactory;
 import org.apache.solr.response.transform.ValueSourceAugmenter;
+import org.apache.solr.search.SolrDocumentFetcher.RetrieveFieldsOptimizer;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -41,6 +42,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Supplier;
 
 /**
  * The default implementation of return fields parsing for Solr.
@@ -68,6 +70,27 @@ public class SolrReturnFields extends ReturnFields {
   protected boolean _wantsAllFields = false;
   protected Map<String,String> renameFields = Collections.emptyMap();
 
+  // Only set currently with the SolrDocumentFetcher.solrDoc method. Primarily used
+  // at this time for testing to ensure we get fields from the expected places.
+  public enum FIELD_SOURCES {
+      NOT_SET, ALL_FROM_DV, ALL_FROM_STORED, MIXED_SOURCES
+  }
+
+  public FIELD_SOURCES getFieldSources() {
+    return fieldSources;
+  }
+
+  public void setFieldSources(FIELD_SOURCES fieldSources) {
+    this.fieldSources = fieldSources;
+  }
+
+  private FIELD_SOURCES fieldSources = FIELD_SOURCES.NOT_SET;
+  // For each individual result list, we need to have a separate fetch optimizer
+  // to use. It's particularly important to keep this list separated during, say,
+  // sub-query transformations.
+  //
+  private RetrieveFieldsOptimizer fetchOptimizer = null;
+
   public SolrReturnFields() {
     _wantsAllFields = true;
   }
@@ -100,6 +123,14 @@ public class SolrReturnFields extends ReturnFields {
     parseFieldList(fl, req);
   }
 
+  public RetrieveFieldsOptimizer getFetchOptimizer(Supplier<RetrieveFieldsOptimizer> supplier) {
+    if (fetchOptimizer == null) {
+      fetchOptimizer = supplier.get();
+    }
+    return fetchOptimizer;
+  }
+
+
   private void parseFieldList(String[] fl, SolrQueryRequest req) {
     _wantsScore = false;
     _wantsAllFields = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/core/src/test/org/apache/solr/response/TestRetrieveFieldsOptimizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/response/TestRetrieveFieldsOptimizer.java b/solr/core/src/test/org/apache/solr/response/TestRetrieveFieldsOptimizer.java
index 6ea0f42..4409efd 100644
--- a/solr/core/src/test/org/apache/solr/response/TestRetrieveFieldsOptimizer.java
+++ b/solr/core/src/test/org/apache/solr/response/TestRetrieveFieldsOptimizer.java
@@ -17,40 +17,632 @@
 
 package org.apache.solr.response;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.DatePointField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TrieDateField;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SolrReturnFields;
+import org.apache.solr.util.RefCounted;
+import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
 
-public class TestRetrieveFieldsOptimizer extends SolrTestCaseJ4{
+import static junit.framework.Assert.fail;
+import static org.apache.lucene.util.LuceneTestCase.random;
+import static org.apache.solr.search.SolrReturnFields.FIELD_SOURCES.ALL_FROM_STORED;
+import static org.apache.solr.search.SolrReturnFields.FIELD_SOURCES.MIXED_SOURCES;
+import static org.apache.solr.search.SolrReturnFields.FIELD_SOURCES.ALL_FROM_DV;
+
+public class TestRetrieveFieldsOptimizer extends SolrTestCaseJ4 {
+
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+
+  @BeforeClass
+  public static void initManagedSchemaCore() throws Exception {
+    // This testing approach means no schema file or per-test temp solr-home!
+    System.setProperty("managed.schema.mutable", "true");
+    System.setProperty("managed.schema.resourceName", "schema-one-field-no-dynamic-field-unique-key.xml");
+    System.setProperty("enable.update.log", "false");
+
+    initCore("solrconfig-managed-schema.xml", "ignoredSchemaName");
+
+    IndexSchema schema = h.getCore().getLatestSchema();
+    setupAllFields();
+
+    h.getCore().setLatestSchema(schema);
+  }
+
+  static String storedNotDvSv = "_s_ndv_sv";
+  static String storedAndDvSv = "_s_dv_sv";
+  static String notStoredDvSv = "_ns_dv_sv";
+
+  static String storedNotDvMv = "_s_ndv_mv";
+  static String storedAndDvMv = "_s_dv_mv";
+  static String notStoredDvMv = "_ns_dv_mv";
+
+  // Each doc needs a field I can use to identify it for value comparison
+  static String idStoredNotDv = "id_s_ndv_sv";
+  static String idNotStoredDv = "id_ns_dv_sv";
+
+  static FieldTypeHolder typesHolder = new FieldTypeHolder();
+
+  static FieldHolder fieldsHolder = new FieldHolder();
+  static Map<String, Map<String, List<String>>> allFieldValuesInput = new HashMap<>();
+
+  //TODO, how to generalize?
+
+  private static void setupAllFields() throws IOException {
+
+    IndexSchema schema = h.getCore().getLatestSchema();
+
+    // Add all the types before the fields.
+    Map<String, Map<String, String>> fieldsToAdd = new HashMap<>();
+
+    // We need our special id fields to find the docs later.
+    typesHolder.addFieldType(schema, idNotStoredDv, RetrieveFieldType.TEST_TYPE.STRING);
+    fieldsToAdd.put(idNotStoredDv, map("stored", "false", "docValues", "true", "multiValued", "false"));
+
+    typesHolder.addFieldType(schema, idStoredNotDv, RetrieveFieldType.TEST_TYPE.STRING);
+    fieldsToAdd.put(idStoredNotDv, map("stored", "true", "docValues", "false", "multiValued", "false"));
+
+    for (RetrieveFieldType.TEST_TYPE type : RetrieveFieldType.solrClassMap.keySet()) {
+      // We happen to be naming the fields and types identically.
+      String myName = type.toString() + storedNotDvSv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "true", "docValues", "false", "multiValued", "false"));
+
+      myName = type.toString() + storedAndDvSv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "true", "docValues", "true", "multiValued", "false"));
+
+      myName = type.toString() + notStoredDvSv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "false", "docValues", "true", "multiValued", "false"));
+
+      myName = type.toString() + storedNotDvMv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "true", "docValues", "false", "multiValued", "true"));
+
+      myName = type.toString() + storedAndDvMv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "true", "docValues", "true", "multiValued", "true"));
+
+      myName = type.toString() + notStoredDvMv;
+      typesHolder.addFieldType(schema, myName, type);
+      fieldsToAdd.put(myName, map("stored", "false", "docValues", "true", "multiValued", "true"));
+    }
+
+    schema = typesHolder.addFieldTypes(schema);
+
+    for (Map.Entry<String, Map<String, String>> ent : fieldsToAdd.entrySet()) {
+      fieldsHolder.addField(schema, ent.getKey(), ent.getKey(), ent.getValue());
+    }
+    schema = fieldsHolder.addFields(schema);
+
+    h.getCore().setLatestSchema(schema);
+
+    // All that setup work and we're only going to add a very few docs!
+    for (int idx = 0; idx < 10; ++idx) {
+      addDocWithAllFields(idx);
+    }
+    assertU(commit());
+    // Now we need to massage the expected values returned based on the docValues type 'cause it's weird.
+    final RefCounted<SolrIndexSearcher> refCounted = h.getCore().getNewestSearcher(true);
+    try {
+      //static Map<String, Map<String, List<String>>>
+      for (Map<String, List<String>> docFieldsEnt : allFieldValuesInput.values()) {
+        for (Map.Entry<String, List<String>> oneField : docFieldsEnt.entrySet()) {
+          RetrieveField field = fieldsHolder.getTestField(oneField.getKey());
+          field.expectedValsAsStrings(refCounted.get().getSlowAtomicReader().getFieldInfos().fieldInfo(field.name),
+              oneField.getValue());
+        }
+      }
+    } finally {
+      refCounted.decref();
+    }
+   }
+
+  static void addDocWithAllFields(int idx) {
+
+    // for each doc, add a doc with all the fields with values and store the expected return.
+    Map<String, List<String>> fieldsExpectedVals = new HashMap<>();
+
+    SolrInputDocument sdoc = new SolrInputDocument();
+    String id = "str" + idx;
+    sdoc.addField("str", id);
+    sdoc.addField(idNotStoredDv, id);
+    fieldsExpectedVals.put(idNotStoredDv, Collections.singletonList(id));
+    sdoc.addField(idStoredNotDv, id);
+    fieldsExpectedVals.put(idStoredNotDv, Collections.singletonList(id));
+
+    for (RetrieveField field : fieldsHolder.fields.values()) {
+      if (field.name.equals(idNotStoredDv) || field.name.equals(idStoredNotDv)) {
+        continue;
+      }
+      List<String> valsAsStrings = field.getValsForField();
+      for (String val : valsAsStrings) {
+        sdoc.addField(field.schemaField.getName(), val);
+      }
+      fieldsExpectedVals.put(field.name, valsAsStrings);
+    }
+
+    allFieldValuesInput.put(id, fieldsExpectedVals);
+    assertU(adoc(sdoc));
+  }
 
   @Test
-  public void testOptimizer() {
-    RetrieveFieldsOptimizer optimizer = new RetrieveFieldsOptimizer(
-        new HashSet<>(Arrays.asList("id", "title")),
-        new HashSet<>()
-    );
-    optimizer.optimize(new HashSet<>(Arrays.asList("id", "title")));
-    assertTrue(optimizer.returnDVFields());
-    assertFalse(optimizer.returnStoredFields());
-
-    optimizer = new RetrieveFieldsOptimizer(
-        new HashSet<>(Arrays.asList("id", "title")),
-        new HashSet<>()
-    );
-    optimizer.optimize(new HashSet<>(Collections.singletonList("title")));
-    assertFalse(optimizer.returnDVFields());
-    assertTrue(optimizer.returnStoredFields());
-
-    optimizer = new RetrieveFieldsOptimizer(
-        null,
-        new HashSet<>(Collections.singletonList("id"))
-    );
-    optimizer.optimize(new HashSet<>(Collections.singletonList("id")));
-    assertNull(optimizer.getStoredFields());
-    assertTrue(optimizer.getDvFields().contains("id"));
+  public void testDocFetcher() throws Exception {
+
+    int numThreads = random().nextInt(3) + 2;
+
+    Thread threads[] = new Thread[numThreads];
+    for (int idx = 0; idx < numThreads; idx++) {
+      threads[idx] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            checkFetchSources(ALL_FROM_DV);
+            checkFetchSources(ALL_FROM_STORED);
+            checkFetchSources(MIXED_SOURCES);
+          } catch (Exception e) {
+            fail("Failed with exception " + e.getMessage());
+          }
+        }
+      };
+      threads[idx].start();
+    }
+    for (int idx = 0; idx < numThreads; idx++) {
+      threads[idx].join();
+    }
+  }
+
+  private void checkFetchSources(SolrReturnFields.FIELD_SOURCES source) throws Exception {
+    String flAll = fieldsHolder.allFields.stream()
+        .map(RetrieveField::getName) // This will call testField.getName()
+        .collect(Collectors.joining(","));
+
+    List<RetrieveField> toCheck = new ArrayList<>();
+    String idField = idNotStoredDv + ",";
+    switch (source) {
+      case ALL_FROM_DV:
+        toCheck = new ArrayList(fieldsHolder.dvNotStoredFields);
+        break;
+      case ALL_FROM_STORED:
+        idField = idStoredNotDv + ",";
+        toCheck = new ArrayList(fieldsHolder.storedNotDvFields);
+        break;
+      case MIXED_SOURCES:
+        toCheck = new ArrayList(fieldsHolder.allFields);
+        break;
+      default:
+        fail("Value passed to checkFetchSources unknown: " + source.toString());
+    }
+
+    // MultiValued fields are _always_ read from stored data.
+    toCheck.removeAll(fieldsHolder.multiValuedFields);
+
+    // At this point, toCheck should be only singleValued fields. Adding in even a single multiValued field should
+    // read stuff from stored.
+    String fl = idField + toCheck.stream()
+        .map(RetrieveField::getName) // This will call testField.getName()
+        .collect(Collectors.joining(","));
+
+    // Even a single multiValued and stored field should cause stored fields to be visited.
+
+    List<Integer> shuffled = Arrays.asList(0, 1, 2);
+    Collections.shuffle(shuffled, random());
+    for (int which : shuffled) {
+      switch (which) {
+        case 0:
+          check(fl, source);
+          break;
+
+        case 1:
+          check(flAll, MIXED_SOURCES);
+          break;
+
+        case 2:
+          List<RetrieveField> toCheckPlusMv = new ArrayList<>(toCheck);
+          toCheckPlusMv.add(fieldsHolder.storedMvFields.get(random().nextInt(fieldsHolder.storedMvFields.size())));
+
+          String flWithMv = idField + toCheckPlusMv.stream()
+              .map(RetrieveField::getName) // This will call testField.getName()
+              .collect(Collectors.joining(","));
+          if (source == ALL_FROM_STORED) {
+            check(flWithMv, ALL_FROM_STORED);
+          } else {
+            check(flWithMv, MIXED_SOURCES);
+          }
+          break;
+        default:
+          fail("Your shuffling should be between 0 and 2, inclusive. It was: " + which);
+      }
+    }
+  }
+
+  // This checks a couple of things:
+  // 1> we got all the values from the place we expected.
+  // 2> all the values we expect are actually returned.
+  //
+  // NOTE: multiValued fields are _NOT_ fetched from docValues by design so we don't have to worry about set semantics
+  //
+  private void check(String flIn, SolrReturnFields.FIELD_SOURCES source) throws Exception {
+    Set<String> setDedupe = new HashSet<>(Arrays.asList(flIn.split(",")));
+    String fl = String.join(",", setDedupe);
+
+    SolrCore core = h.getCore();
+
+    SolrQueryRequest req = lrf.makeRequest("q", "*:*", CommonParams.FL, fl);
+    SolrQueryResponse rsp = h.queryAndResponse("", req);
+
+    BinaryQueryResponseWriter writer = (BinaryQueryResponseWriter) core.getQueryResponseWriter("javabin");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    writer.write(baos, req, rsp);
+
+    // This is really the main point!
+    assertEquals("We didn't get the values from the expected places! ",
+        source, ((SolrReturnFields) rsp.returnFields).getFieldSources());
+
+    NamedList res;
+    try (JavaBinCodec jbc = new JavaBinCodec()) {
+      res = (NamedList) jbc.unmarshal(new ByteArrayInputStream(baos.toByteArray()));
+    }
+    SolrDocumentList docs = (SolrDocumentList) res.get("response");
+    for (Object doc : docs) {
+      SolrDocument sdoc = (SolrDocument) doc;
+      // Check that every (and only) the fields in the fl param were fetched and the values are as expected.
+      // Since each doc has the same fields, we don't need to find the special doc.
+      String[] requestedFields = fl.split(",");
+      assertEquals("Should have exactly as many fields as requested, ", sdoc.getFieldNames().size(), requestedFields.length);
+
+      String id = (String) sdoc.get(idNotStoredDv);
+      if (id == null) {
+        id = (String) sdoc.get(idStoredNotDv);
+      }
+      Map<String, List<String>> expected = allFieldValuesInput.get(id);
+      for (String field : requestedFields) {
+        Object[] docVals = sdoc.getFieldValues(field).toArray();
+        RetrieveField testField = fieldsHolder.getTestField(field);
+        List<String> expectedVals = expected.get(field);
+        assertEquals("Returned fields should have the expected number of entries", docVals.length, expectedVals.size());
+        for (int idx = 0; idx < docVals.length; ++idx) {
+          assertEquals("Values should be identical and exactly in order. ", expectedVals.get(idx), testField.getValAsString(docVals[idx]));
+        }
+      }
+
+    }
+    req.close();
+  }
+}
+
+class FieldTypeHolder {
+
+  Map<String, RetrieveFieldType> testTypes = new HashMap<>();
+
+  void addFieldType(IndexSchema schema, String name, RetrieveFieldType.TEST_TYPE type) {
+    testTypes.put(name, new RetrieveFieldType(schema, name, type));
+  }
+
+  IndexSchema addFieldTypes(IndexSchema schema) {
+    List<FieldType> typesToAdd = new ArrayList<>();
+    for (RetrieveFieldType testType : testTypes.values()) {
+      typesToAdd.add(testType.getFieldType());
+    }
+    return schema.addFieldTypes(typesToAdd, false);
+  }
 
+  RetrieveFieldType getTestType(String name) {
+    return testTypes.get(name);
   }
 }
+
+class RetrieveFieldType {
+  final String name;
+  final FieldType solrFieldType;
+  final TEST_TYPE testType;
+  final String solrTypeClass;
+
+  static enum TEST_TYPE {
+    TINT, TLONG, TFLOAT, TDOUBLE, TDATE,
+    PINT, PLONG, PFLOAT, PDOUBLE, PDATE,
+    STRING, BOOL
+  }
+
+  static final Map<TEST_TYPE, String> solrClassMap = Collections.unmodifiableMap(Stream.of(
+      new SimpleEntry<>(TEST_TYPE.TINT, "solr.TrieIntField"),
+      new SimpleEntry<>(TEST_TYPE.TLONG, "solr.TrieLongField"),
+      new SimpleEntry<>(TEST_TYPE.TFLOAT, "solr.TrieFloatField"),
+      new SimpleEntry<>(TEST_TYPE.TDOUBLE, "solr.TrieDoubleField"),
+      new SimpleEntry<>(TEST_TYPE.TDATE, "solr.TrieDateField"),
+      new SimpleEntry<>(TEST_TYPE.PINT, "solr.IntPointField"),
+      new SimpleEntry<>(TEST_TYPE.PLONG, "solr.LongPointField"),
+      new SimpleEntry<>(TEST_TYPE.PFLOAT, "solr.FloatPointField"),
+      new SimpleEntry<>(TEST_TYPE.PDOUBLE, "solr.DoublePointField"),
+      new SimpleEntry<>(TEST_TYPE.PDATE, "solr.DatePointField"),
+      new SimpleEntry<>(TEST_TYPE.STRING, "solr.StrField"),
+      new SimpleEntry<>(TEST_TYPE.BOOL, "solr.BoolField"))
+      .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+
+  RetrieveFieldType(IndexSchema schema, String name, TEST_TYPE type) {
+    this.name = name;
+    Map<String, String> opts = new HashMap<>();
+    opts.put("name", name);
+    this.solrTypeClass = solrClassMap.get(type);
+    opts.put("class", solrTypeClass);
+    solrFieldType = schema.newFieldType(name, solrTypeClass, opts);
+    this.testType = type;
+  }
+
+  FieldType getFieldType() {
+    return solrFieldType;
+  }
+
+  String getSolrTypeClass() {
+    return solrTypeClass;
+  }
+}
+
+class FieldHolder {
+
+  Map<String, RetrieveField> fields = new HashMap<>();
+
+  void addField(IndexSchema schema, String name, String type, Map<String, String> opts) {
+    fields.put(name, new RetrieveField(schema, name, type, opts));
+  }
+
+  List<RetrieveField> dvNotStoredFields = new ArrayList<>();
+  List<RetrieveField> storedNotDvFields = new ArrayList<>();
+  List<RetrieveField> multiValuedFields = new ArrayList<>();
+  List<RetrieveField> storedAndDvFields = new ArrayList<>();
+  List<RetrieveField> storedMvFields = new ArrayList<>();
+  List<RetrieveField> allFields = new ArrayList<>();
+
+  IndexSchema addFields(IndexSchema schema) {
+
+    List<SchemaField> fieldsToAdd = new ArrayList<>();
+
+    for (RetrieveField field : fields.values()) {
+      allFields.add(field);
+      SchemaField schemaField = field.schemaField;
+      fieldsToAdd.add(schemaField);
+      if (schemaField.multiValued()) {
+        multiValuedFields.add(field);
+      }
+      if (schemaField.hasDocValues() && schemaField.stored() == false) {
+        dvNotStoredFields.add(field);
+      }
+      if (schemaField.hasDocValues() == false && schemaField.stored()) {
+        storedNotDvFields.add(field);
+      }
+      if (schemaField.hasDocValues() && schemaField.stored()) {
+        storedAndDvFields.add(field);
+      }
+      if (schemaField.stored() && schemaField.multiValued()) {
+        storedMvFields.add(field);
+      }
+    }
+    return schema.addFields(fieldsToAdd, Collections.emptyMap(), false);
+  }
+
+  RetrieveField getTestField(String field) {
+    return fields.get(field);
+  }
+}
+
+class RetrieveField {
+  final String name;
+  final String type;
+  final SchemaField schemaField;
+  final RetrieveFieldType testFieldType;
+
+  RetrieveField(IndexSchema schema, String name, String type, Map<String, String> opts) {
+
+    Map<String, String> fullOpts = new HashMap<>(opts);
+    fullOpts.put("name", name);
+    fullOpts.put("type", type);
+
+    this.name = name;
+    this.type = type;
+    this.schemaField = schema.newField(name, type, opts);
+    this.testFieldType = TestRetrieveFieldsOptimizer.typesHolder.getTestType(type);
+
+  }
+
+  String getValAsString(Object val) {
+
+    FieldType fieldType = schemaField.getType();
+
+    //Why do mutliValued date fields get here as Strings whereas single-valued fields are Dates?
+    // Why do BoolFields sometimes get here as "F" or "T"?
+    if (val instanceof String) {
+      if (fieldType instanceof TrieDateField || fieldType instanceof DatePointField) {
+        long lVal = Long.parseLong((String) val);
+        return (new Date(lVal).toInstant().toString());
+      }
+      if (fieldType instanceof BoolField) {
+        if (val.equals("F")) return "false";
+        if (val.equals("T")) return "true";
+      }
+      return (String) val;
+    }
+    if (fieldType instanceof TrieDateField || fieldType instanceof DatePointField) {
+      return ((Date) val).toInstant().toString();
+    }
+
+    return val.toString();
+  }
+
+  String getName() {
+    return schemaField.getName();
+  }
+
+  static String chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890";
+
+  private String randString() {
+    StringBuilder sb = new StringBuilder();
+    sb.setLength(0);
+
+    for (int idx = 0; idx < 10; ++idx) {
+      sb.append(chars.charAt(random().nextInt(chars.length())));
+    }
+    return sb.toString();
+  }
+
+  private String randDate() {
+    return new Date(Math.abs(random().nextLong()) % 3_000_000_000_000L).toInstant().toString();
+  }
+
+  List<String> getValsForField() {
+    List<String> valsAsStrings = new ArrayList<>();
+    switch (testFieldType.getSolrTypeClass()) {
+      case "solr.TrieIntField":
+      case "solr.TrieLongField":
+      case "solr.IntPointField":
+      case "solr.LongPointField":
+        valsAsStrings.add(Integer.toString(random().nextInt(10_000)));
+        if (schemaField.multiValued() == false) break;
+        for (int idx = 0; idx < random().nextInt(5); ++idx) {
+          valsAsStrings.add(Integer.toString(random().nextInt(10_000)));
+        }
+        break;
+
+      case "solr.TrieFloatField":
+      case "solr.TrieDoubleField":
+      case "solr.FloatPointField":
+      case "solr.DoublePointField":
+        valsAsStrings.add(Float.toString(random().nextFloat()));
+        if (schemaField.multiValued() == false) break;
+        for (int idx = 0; idx < random().nextInt(5); ++idx) {
+          valsAsStrings.add(Float.toString(random().nextFloat()));
+        }
+        break;
+
+      case "solr.TrieDateField":
+      case "solr.DatePointField":
+        valsAsStrings.add(randDate());
+        if (schemaField.multiValued() == false) break;
+        for (int idx = 0; idx < random().nextInt(5); ++idx) {
+          valsAsStrings.add(randDate());
+        }
+        break;
+
+      case "solr.StrField":
+        valsAsStrings.add(randString());
+        if (schemaField.multiValued() == false) break;
+        for (int idx = 0; idx < random().nextInt(5); ++idx) {
+          valsAsStrings.add(randString());
+        }
+        break;
+
+      case "solr.BoolField":
+        valsAsStrings.add(Boolean.toString(random().nextBoolean()));
+        if (schemaField.multiValued() == false) break;
+        for (int idx = 0; idx < random().nextInt(5); ++idx) {
+          valsAsStrings.add(Boolean.toString(random().nextBoolean()));
+        }
+        break;
+
+      default:
+        fail("Found no case for field " + name + " type " + type);
+        break;
+    }
+    // There are tricky cases with multiValued fields that are sometimes fetched from docValues that obey set
+    // semantics so be sure we include at least one duplicate in a multValued field sometimes
+    if (random().nextBoolean() && valsAsStrings.size() > 1) {
+      valsAsStrings.add(valsAsStrings.get(random().nextInt(valsAsStrings.size())));
+
+    }
+
+    return valsAsStrings;
+  }
+
+  void expectedValsAsStrings(final FieldInfo info, List<String> valsAsStrings) {
+    if (schemaField.stored() || schemaField.multiValued() == false) {
+      return ;
+    }
+
+    switch (info.getDocValuesType()) {
+      case NONE: // These three types are single values, just return.
+      case NUMERIC:
+      case BINARY: // here for completeness, really doesn't make sense.
+        return;
+
+      case SORTED_NUMERIC: // Can have multiple, identical values. This was a surprise to me.
+        break;
+
+      case SORTED_SET: // Obey set semantics.
+      case SORTED:
+        Set<String> uniq = new TreeSet<>(valsAsStrings);
+        valsAsStrings.clear();
+        valsAsStrings.addAll(uniq);
+        break;
+    }
+
+    // Now order them if string-based comparison isn't reasonable
+    switch (testFieldType.getSolrTypeClass()) {
+      case "solr.TrieIntField":
+      case "solr.TrieLongField":
+
+        Collections.sort(valsAsStrings, Comparator.comparingInt(Integer::parseInt));
+        break;
+      case "solr.IntPointField":
+      case "solr.LongPointField":
+        Collections.sort(valsAsStrings, Comparator.comparingLong(Long::parseLong));
+        break;
+
+      case "solr.TrieFloatField":
+      case "solr.FloatPointField":
+      case "solr.TrieDoubleField":
+      case "solr.DoublePointField":
+        Collections.sort(valsAsStrings, Comparator.comparingDouble(Double::parseDouble));
+        break;
+
+      case "solr.TrieDateField":
+      case "solr.DatePointField":
+      case "solr.StrField":
+      case "solr.BoolField":
+        Collections.sort(valsAsStrings);
+        break;
+
+      default:
+        fail("Found no case for field " + name + " type " + type);
+        break;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82c64af8/solr/solr-ref-guide/src/docvalues.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/docvalues.adoc b/solr/solr-ref-guide/src/docvalues.adoc
index d25a8c8..1502302 100644
--- a/solr/solr-ref-guide/src/docvalues.adoc
+++ b/solr/solr-ref-guide/src/docvalues.adoc
@@ -42,26 +42,26 @@ If you have already indexed data into your Solr index, you will need to complete
 
 DocValues are only available for specific field types. The types chosen determine the underlying Lucene docValue type that will be used. The available Solr field types are:
 
-* `StrField` and `UUIDField`:
+* `StrField`, and `UUIDField`:
 ** If the field is single-valued (i.e., multi-valued is false), Lucene will use the `SORTED` type.
-** If the field is multi-valued, Lucene will use the `SORTED_SET` type.
+** If the field is multi-valued, Lucene will use the `SORTED_SET` type. Entries are kept in sorted order an duplicates are removed.
 * `BoolField`:
 ** If the field is single-valued (i.e., multi-valued is false), Lucene will use the `SORTED` type.
-** If the field is multi-valued, Lucene will use the `SORTED_BINARY` type.
+** If the field is multi-valued, Lucene will use the `SORTED_SET` type. Entries are kept in sorted order an duplicates are removed.
 * Any `*PointField` Numeric or Date fields, `EnumFieldType`, and `CurrencyFieldType`:
 ** If the field is single-valued (i.e., multi-valued is false), Lucene will use the `NUMERIC` type.
-** If the field is multi-valued, Lucene will use the `SORTED_NUMERIC` type.
+** If the field is multi-valued, Lucene will use the `SORTED_NUMERIC` type. Entries are kept in sorted order and duplicates are kept.
 * Any of the deprecated `Trie*` Numeric or Date fields, `EnumField` and `CurrencyField`:
 ** If the field is single-valued (i.e., multi-valued is false), Lucene will use the `NUMERIC` type.
-** If the field is multi-valued, Lucene will use the `SORTED_SET` type.
+** If the field is multi-valued, Lucene will use the `SORTED_SET` type. Entries are kept in sorted order an duplicates are removed.
 
 These Lucene types are related to how the {lucene-javadocs}/core/org/apache/lucene/index/DocValuesType.html[values are sorted and stored].
 
-There is an additional configuration option available, which is to modify the `docValuesFormat` <<field-type-definitions-and-properties.adoc#docvaluesformat,used by the field type>>. The default implementation employs a mixture of loading some things into memory and keeping some on disk. In some cases, however, you may choose to specify an alternative {lucene-javadocs}/core/org/apache/lucene/codecs/DocValuesFormat.html[DocValuesFormat implementation]. For example, you could choose to keep everything in memory by specifying `docValuesFormat="Memory"` on a field type:
+There is an additional configuration option available, which is to modify the `docValuesFormat` <<field-type-definitions-and-properties.adoc#docvaluesformat,used by the field type>>. The default implementation employs a mixture of loading some things into memory and keeping some on disk. In some cases, however, you may choose to specify an alternative {lucene-javadocs}/core/org/apache/lucene/codecs/DocValuesFormat.html[DocValuesFormat implementation]. For example, you could choose to keep everything in memory by specifying `docValuesFormat="Direct"` on a field type:
 
 [source,xml]
 ----
-<fieldType name="string_in_mem_dv" class="solr.StrField" docValues="true" docValuesFormat="Memory" />
+<fieldType name="string_in_mem_dv" class="solr.StrField" docValues="true" docValuesFormat="Direct" />
 ----
 
 Please note that the `docValuesFormat` option may change in future releases.
@@ -79,11 +79,11 @@ If `docValues="true"` for a field, then DocValues will automatically be used any
 
 Field values retrieved during search queries are typically returned from stored values. However, non-stored docValues fields will be also returned along with other stored fields when all fields (or pattern matching globs) are specified to be returned (e.g., "`fl=*`") for search queries depending on the effective value of the `useDocValuesAsStored` parameter for each field. For schema versions >= 1.6, the implicit default is `useDocValuesAsStored="true"`. See <<field-type-definitions-and-properties.adoc#field-type-definitions-and-properties,Field Type Definitions and Properties>> & <<defining-fields.adoc#defining-fields,Defining Fields>> for more details.
 
-When `useDocValuesAsStored="false"`, non-stored DocValues fields can still be explicitly requested by name in the <<common-query-parameters.adoc#fl-field-list-parameter,fl param>>, but will not match glob patterns (`"*"`). Note that returning DocValues along with "regular" stored fields at query time has performance implications that stored fields may not because DocValues are column-oriented and may therefore incur additional cost to retrieve for each returned document. Also note that while returning non-stored fields from DocValues, the values of a multi-valued field are returned in sorted order (and not insertion order). If you require the multi-valued fields to be returned in the original insertion order, then make your multi-valued field as stored (such a change requires re-indexing).
+When `useDocValuesAsStored="false"`, non-stored DocValues fields can still be explicitly requested by name in the <<common-query-parameters.adoc#fl-field-list-parameter,fl param>>, but will not match glob patterns (`"*"`). Note that returning DocValues along with "regular" stored fields at query time has performance implications that stored fields may not because DocValues are column-oriented and may therefore incur additional cost to retrieve for each returned document. Also note that while returning non-stored fields from DocValues, the values of a multi-valued field are returned in sorted order rather than insertion order and may have duplicates removed, see above. If you require the multi-valued fields to be returned in the original insertion order, then make your multi-valued field as stored (such a change requires re-indexing).
 
 In cases where the query is returning _only_ docValues fields performance may improve since returning stored fields requires disk reads and decompression whereas returning docValues fields in the fl list only requires memory access.
 
 When retrieving fields from their docValues form (using the <<exporting-result-sets.adoc#exporting-result-sets,/export handler>>, <<streaming-expressions.adoc#streaming-expressions,streaming expressions>> or if the field is requested in the `fl` parameter), two important differences between regular stored fields and docValues fields must be understood:
 
-1.  Order is _not_ preserved. For simply retrieving stored fields, the insertion order is the return order. For docValues, it is the _sorted_ order.
-2.  For field types using `SORTED_SET`, multiple identical entries are collapsed into a single value. Thus if I insert values 4, 5, 2, 4, 1, my return will be 1, 2, 4, 5.
+1.  Order is _not_ preserved. When retrieving stored fields, the insertion order is the return order. For docValues, it is the _sorted_ order.
+2.  For field types using `SORTED_SET` (see above), multiple identical entries are collapsed into a single value. Thus if I insert values 4, 5, 2, 4, 1, my return will be 1, 2, 4, 5.