You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/03/23 11:50:01 UTC

[14/46] lucene-solr:jira/solr-9959: SOLR-10286: large fields. And refactored FieldType.checkSchemaField to call a new checkSupportsDocValues()

SOLR-10286: large fields.
And refactored FieldType.checkSchemaField to call a new checkSupportsDocValues()


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

Branch: refs/heads/jira/solr-9959
Commit: 2502af9f3fa25a1b724400af61bf74102f2475dd
Parents: f8831ce
Author: David Smiley <ds...@apache.org>
Authored: Thu Mar 16 14:58:59 2017 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Thu Mar 16 14:58:59 2017 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   6 +
 .../apache/solr/schema/ICUCollationField.java   |   5 +-
 .../handler/component/RealTimeGetComponent.java |  36 ++-
 .../transform/BaseEditorialTransformer.java     |  36 +--
 .../org/apache/solr/schema/BinaryField.java     |   9 +
 .../java/org/apache/solr/schema/BoolField.java  |   3 -
 .../org/apache/solr/schema/CollationField.java  |   5 +-
 .../java/org/apache/solr/schema/EnumField.java  |   7 -
 .../org/apache/solr/schema/FieldProperties.java |  43 +--
 .../java/org/apache/solr/schema/FieldType.java  |  20 +-
 .../solr/schema/LatLonPointSpatialField.java    |   3 +-
 .../java/org/apache/solr/schema/PointField.java |   4 -
 .../apache/solr/schema/PrimitiveFieldType.java  |   4 +
 .../org/apache/solr/schema/SchemaField.java     |   7 +-
 .../java/org/apache/solr/schema/StrField.java   |   4 -
 .../java/org/apache/solr/schema/TrieField.java  |   3 -
 .../apache/solr/search/SolrIndexSearcher.java   | 268 ++++++++++++-------
 .../org/apache/solr/update/DocumentBuilder.java |   7 +-
 .../conf/schema-unifiedhighlight.xml            |   4 +-
 .../conf/solrconfig-managed-schema.xml          |   2 +-
 .../apache/solr/cloud/TestRandomFlRTGCloud.java |   2 +-
 .../highlight/TestUnifiedSolrHighlighter.java   |  12 +-
 .../apache/solr/schema/SortableBinaryField.java |   3 +-
 .../org/apache/solr/search/LargeFieldTest.java  | 141 ++++++++++
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +-
 25 files changed, 448 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7ec0c52..af151ed 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -197,6 +197,12 @@ New Features
 
 * SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode (Joel Bernstein)
 
+* SOLR-10286: string/text fields may now declare themselves as large="true" in the schema.  Large fields are always
+  lazy loaded and will only take up space in the document cache if the actual value is < 512KB.  This option
+  requires "stored" and must not be multiValued.  It's intended for fields that might have very large values so that
+  they don't get cached in memory. (David Smiley)
+
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
index 7d9e1c7..79cb6b3 100644
--- a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
+++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
@@ -277,10 +277,9 @@ public class ICUCollationField extends FieldType {
       return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive);
     }
   }
-  
+
   @Override
-  public void checkSchemaField(SchemaField field) {
-    // no-op
+  protected void checkSupportsDocValues() { // we support DocValues
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index 123abea..900c787 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -690,24 +690,52 @@ public class RealTimeGetComponent extends SearchComponent
           List<Object> vals = new ArrayList<>();
           if (f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
             // SORTED_NUMERICS store sortable bits version of the value, need to retrieve the original
-            vals.add(sf.getType().toObject(f));
+            vals.add(sf.getType().toObject(f)); // (will materialize by side-effect)
           } else {
-            vals.add( f );
+            vals.add( materialize(f) );
           }
           out.setField( f.name(), vals );
         }
         else{
-          out.setField( f.name(), f );
+          out.setField( f.name(), materialize(f) );
         }
       }
       else {
-        out.addField( f.name(), f );
+        out.addField( f.name(), materialize(f) );
       }
     }
     return out;
   }
 
   /**
+   * Ensure we don't have {@link org.apache.lucene.document.LazyDocument.LazyField} or equivalent.
+   * It can pose problems if the searcher is about to be closed and we haven't fetched a value yet.
+   */
+  private static IndexableField materialize(IndexableField in) {
+    if (in instanceof Field) { // already materialized
+      return in;
+    }
+    return new ClonedField(in);
+  }
+
+  private static class ClonedField extends Field { // TODO Lucene Field has no copy constructor; maybe it should?
+    ClonedField(IndexableField in) {
+      super(in.name(), in.fieldType());
+      this.fieldsData = in.numericValue();
+      if (this.fieldsData == null) {
+        this.fieldsData = in.binaryValue();
+        if (this.fieldsData == null) {
+          this.fieldsData = in.stringValue();
+          if (this.fieldsData == null) {
+            // fallback:
+            assert false : in; // unexpected
+          }
+        }
+      }
+    }
+  }
+
+  /**
    * Converts a SolrInputDocument to SolrDocument, using an IndexSchema instance. 
    * @lucene.experimental
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java b/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java
index 80e0b61..bc5fb65 100644
--- a/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java
+++ b/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java
@@ -17,12 +17,12 @@
 package org.apache.solr.response.transform;
 
 
-import org.apache.lucene.document.Field;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexableField;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.schema.FieldType;
 
-import java.util.Set;
-
 /**
  *
  *
@@ -40,8 +40,7 @@ public abstract class BaseEditorialTransformer extends DocTransformer {
   }
 
   @Override
-  public String getName()
-  {
+  public String getName() {
     return name;
   }
 
@@ -61,22 +60,15 @@ public abstract class BaseEditorialTransformer extends DocTransformer {
   protected abstract Set<String> getIdSet();
 
   protected String getKey(SolrDocument doc) {
-    String key;
-    Object field = doc.get(idFieldName);
-    final Number n;
-    if (field instanceof Field) {
-      n = ((Field) field).numericValue();
-    } else {
-      n = null;
-    }
-    if (n != null) {
-      key = n.toString();
-      key = ft.readableToIndexed(key);
-    } else if (field instanceof Field){
-      key = ((Field)field).stringValue();
-    } else {
-      key = field.toString();
+    Object obj = doc.get(idFieldName);
+    if (obj instanceof IndexableField) {
+      IndexableField f = (IndexableField) obj;
+      Number n = f.numericValue();
+      if (n != null) {
+        return ft.readableToIndexed(n.toString());
+      }
+      return ft.readableToIndexed(f.stringValue());
     }
-    return key;
+    throw new AssertionError("Expected an IndexableField but got: " + obj.getClass());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/BinaryField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/BinaryField.java b/solr/core/src/java/org/apache/solr/schema/BinaryField.java
index d1882b1..a0e7d05 100644
--- a/solr/core/src/java/org/apache/solr/schema/BinaryField.java
+++ b/solr/core/src/java/org/apache/solr/schema/BinaryField.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Base64;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.uninverting.UninvertingReader.Type;
@@ -34,6 +35,14 @@ public class BinaryField extends FieldType  {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  @Override
+  public void checkSchemaField(SchemaField field) {
+    super.checkSchemaField(field);
+    if (field.isLarge()) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; not supported (yet)");
+    }
+  }
+
   private String toBase64String(ByteBuffer buf) {
     return Base64.byteArrayToBase64(buf.array(), buf.position(), buf.limit()-buf.position());
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/BoolField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/BoolField.java b/solr/core/src/java/org/apache/solr/schema/BoolField.java
index 7d5c0d7..607e384 100644
--- a/solr/core/src/java/org/apache/solr/schema/BoolField.java
+++ b/solr/core/src/java/org/apache/solr/schema/BoolField.java
@@ -207,9 +207,6 @@ public class BoolField extends PrimitiveFieldType {
     return Collections.singletonList(fval);
   }
 
-  @Override
-  public void checkSchemaField(final SchemaField field) {
-  }
 }
 
 // TODO - this can be much more efficient - use FixedBitSet or Bits

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/CollationField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/CollationField.java b/solr/core/src/java/org/apache/solr/schema/CollationField.java
index a498c01..c6a4f81 100644
--- a/solr/core/src/java/org/apache/solr/schema/CollationField.java
+++ b/solr/core/src/java/org/apache/solr/schema/CollationField.java
@@ -247,10 +247,9 @@ public class CollationField extends FieldType {
       return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive);
     }
   }
-  
+
   @Override
-  public void checkSchemaField(SchemaField field) {
-    // no-op
+  protected void checkSupportsDocValues() { // we support DocValues
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/EnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java
index 2e73f74..3127262 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java
@@ -281,13 +281,6 @@ public class EnumField extends PrimitiveFieldType {
    * {@inheritDoc}
    */
   @Override
-  public void checkSchemaField(SchemaField field) {
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
   public String readableToIndexed(String val) {
     if (val == null)
       return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
index 2b1a8bb..6762345 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
@@ -30,29 +30,30 @@ public abstract class FieldProperties {
   // many of the variables are independent or semi-independent.
 
   // bit values for boolean field properties.
-  protected final static int INDEXED             = 0x00000001;
-  protected final static int TOKENIZED           = 0x00000002;
-  protected final static int STORED              = 0x00000004;
-  protected final static int BINARY              = 0x00000008;
-  protected final static int OMIT_NORMS          = 0x00000010;
-  protected final static int OMIT_TF_POSITIONS   = 0x00000020;
-  protected final static int STORE_TERMVECTORS   = 0x00000040;
-  protected final static int STORE_TERMPOSITIONS = 0x00000080;
-  protected final static int STORE_TERMOFFSETS   = 0x00000100;
-
-
-  protected final static int MULTIVALUED         = 0x00000200;
-  protected final static int SORT_MISSING_FIRST  = 0x00000400;
-  protected final static int SORT_MISSING_LAST   = 0x00000800;
+  protected final static int INDEXED             = 0b1;
+  protected final static int TOKENIZED           = 0b10;
+  protected final static int STORED              = 0b100;
+  protected final static int BINARY              = 0b1000;
+  protected final static int OMIT_NORMS          = 0b10000;
+  protected final static int OMIT_TF_POSITIONS   = 0b100000;
+  protected final static int STORE_TERMVECTORS   = 0b1000000;
+  protected final static int STORE_TERMPOSITIONS = 0b10000000;
+  protected final static int STORE_TERMOFFSETS   = 0b100000000;
+
+
+  protected final static int MULTIVALUED         = 0b1000000000;
+  protected final static int SORT_MISSING_FIRST  = 0b10000000000;
+  protected final static int SORT_MISSING_LAST   = 0b100000000000;
   
-  protected final static int REQUIRED            = 0x00001000;
-  protected final static int OMIT_POSITIONS      = 0x00002000;
+  protected final static int REQUIRED            = 0b1000000000000;
+  protected final static int OMIT_POSITIONS      = 0b10000000000000;
 
-  protected final static int STORE_OFFSETS       = 0x00004000;
-  protected final static int DOC_VALUES          = 0x00008000;
+  protected final static int STORE_OFFSETS       = 0b100000000000000;
+  protected final static int DOC_VALUES          = 0b1000000000000000;
 
-  protected final static int STORE_TERMPAYLOADS  = 0x00010000;
-  protected final static int USE_DOCVALUES_AS_STORED  = 0x00020000;
+  protected final static int STORE_TERMPAYLOADS  = 0b10000000000000000;
+  protected final static int USE_DOCVALUES_AS_STORED  = 0b100000000000000000;
+  protected final static int LARGE_FIELD         = 0b1000000000000000000;
 
   static final String[] propertyNames = {
           "indexed", "tokenized", "stored",
@@ -60,7 +61,7 @@ public abstract class FieldProperties {
           "termVectors", "termPositions", "termOffsets",
           "multiValued",
           "sortMissingFirst","sortMissingLast","required", "omitPositions",
-          "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored"
+          "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored", "large"
   };
 
   static final Map<String,Integer> propertyMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 67b7be7..016e166 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -791,17 +791,27 @@ public abstract class FieldType extends FieldProperties {
    *
    * <p>
    * This method is called by the <code>SchemaField</code> constructor to 
-   * check that its initialization does not violate any fundemental 
-   * requirements of the <code>FieldType</code>.  The default implementation 
-   * does nothing, but subclasses may chose to throw a {@link SolrException}  
+   * check that its initialization does not violate any fundamental
+   * requirements of the <code>FieldType</code>.
+   * Subclasses may choose to throw a {@link SolrException}
    * if invariants are violated by the <code>SchemaField.</code>
    * </p>
    */
   public void checkSchemaField(final SchemaField field) {
-    // override if your field type supports doc values
     if (field.hasDocValues()) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " does not support doc values");
+      checkSupportsDocValues();
     }
+    if (field.isLarge() && field.multiValued()) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; can't support multiValued");
+    }
+    if (field.isLarge() && getNumberType() != null) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; can't support numerics");
+    }
+  }
+
+  /** Called by {@link #checkSchemaField(SchemaField)} if the field has docValues. By default none do. */
+  protected void checkSupportsDocValues() {
+    throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " does not support doc values");
   }
 
   public static final String TYPE = "type";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
index c09856a..8ed5484 100644
--- a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
@@ -57,8 +57,7 @@ public class LatLonPointSpatialField extends AbstractSpatialFieldType implements
   // TODO handle polygons
 
   @Override
-  public void checkSchemaField(SchemaField field) {
-    // override because if we didn't, FieldType will complain about docValues not being supported (we do support it)
+  protected void checkSupportsDocValues() { // we support DocValues
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/PointField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PointField.java b/solr/core/src/java/org/apache/solr/schema/PointField.java
index 6c75105..91d3eff 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointField.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointField.java
@@ -244,8 +244,4 @@ public abstract class PointField extends NumericFieldType {
 
   protected abstract StoredField getStoredField(SchemaField sf, Object value);
 
-  @Override
-  public void checkSchemaField(final SchemaField field) {
-    // PointFields support DocValues
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
index a920f35..9d9da47 100644
--- a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java
@@ -33,4 +33,8 @@ public abstract class PrimitiveFieldType extends FieldType {
       properties |= OMIT_NORMS;
     }
   }
+
+  @Override
+  protected void checkSupportsDocValues() { // primitive types support DocValues
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/SchemaField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
index e690a13..1e18ee0 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -66,7 +66,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
     args = prototype.args;
   }
 
- /** Create a new SchemaField with the given name and type,
+  /** Create a new SchemaField with the given name and type,
    * and with the specified properties.  Properties are *not*
    * inherited from the type in this case, so users of this
    * constructor should derive the properties from type.getSolrProperties()
@@ -106,7 +106,8 @@ public final class SchemaField extends FieldProperties implements IndexableField
   public boolean multiValued() { return (properties & MULTIVALUED)!=0; }
   public boolean sortMissingFirst() { return (properties & SORT_MISSING_FIRST)!=0; }
   public boolean sortMissingLast() { return (properties & SORT_MISSING_LAST)!=0; }
-  public boolean isRequired() { return required; } 
+  public boolean isRequired() { return required; }
+  public boolean isLarge() { return (properties & LARGE_FIELD)!=0;}
   public Map<String,?> getArgs() { return Collections.unmodifiableMap(args); }
 
   // things that should be determined by field type, not set as options
@@ -240,7 +241,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
     // that depend on that.
     //
     if (on(falseProps,STORED)) {
-      int pp = STORED | BINARY;
+      int pp = STORED | BINARY | LARGE_FIELD;
       if (on(pp,trueProps)) {
         throw new RuntimeException("SchemaField: " + name + " conflicting stored field options:" + props);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/StrField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/StrField.java b/solr/core/src/java/org/apache/solr/schema/StrField.java
index 0b1576b..3294b04 100644
--- a/solr/core/src/java/org/apache/solr/schema/StrField.java
+++ b/solr/core/src/java/org/apache/solr/schema/StrField.java
@@ -96,10 +96,6 @@ public class StrField extends PrimitiveFieldType {
   }
 
   @Override
-  public void checkSchemaField(SchemaField field) {
-  }
-
-  @Override
   public Object marshalSortValue(Object value) {
     return marshalStringSortValue(value);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/schema/TrieField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java
index 46240c9..e7a33bd 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java
@@ -690,9 +690,6 @@ public class TrieField extends NumericFieldType {
     return null;
   }
 
-  @Override
-  public void checkSchemaField(final SchemaField field) {
-  }
 }
 
 class TrieDateFieldSource extends LongFieldSource {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 19e47d0..83df60f 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -18,6 +18,7 @@ package org.apache.solr.search;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.io.Reader;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
@@ -38,63 +39,16 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.collect.Iterables;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.document.LazyDocument;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.ExitableDirectoryReader;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiPostingsEnum;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.StoredFieldVisitor.Status;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.*;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.CollectionStatistics;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.EarlyTerminatingSortingCollector;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.LeafCollector;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.MultiCollector;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermStatistics;
-import org.apache.lucene.search.TimeLimitingCollector;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.TopDocsCollector;
-import org.apache.lucene.search.TopFieldCollector;
-import org.apache.lucene.search.TopFieldDocs;
-import org.apache.lucene.search.TopScoreDocCollector;
-import org.apache.lucene.search.TotalHitCountCollector;
-import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -134,8 +88,6 @@ import org.apache.solr.update.SolrIndexConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Iterables;
-
 /**
  * SolrIndexSearcher adds schema awareness and caching functionality over {@link IndexSearcher}.
  *
@@ -192,7 +144,12 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   /** Contains the names/patterns of all docValues=true,stored=false fields, excluding those that are copyField targets in the schema. */
   private final Set<String> nonStoredDVsWithoutCopyTargets;
 
-  private Collection<String> storedHighlightFieldNames;
+  private static int largeValueLengthCacheThreshold = Integer.getInteger("solr.largeField.cacheThreshold", 512 * 1024); // internal setting
+
+  private final Set<String> largeFields;
+
+  private Collection<String> storedHighlightFieldNames; // lazy populated; use getter
+
   private DirectoryFactory directoryFactory;
 
   private final LeafReader leafReader;
@@ -204,6 +161,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
   private final NamedList<Object> readerStats;
 
+
   private static DirectoryReader getReader(SolrCore core, SolrIndexConfig config, DirectoryFactory directoryFactory,
       String path) throws IOException {
     final Directory dir = directoryFactory.get(path, DirContext.DEFAULT, config.lockType);
@@ -367,11 +325,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     final Set<String> nonStoredDVsUsedAsStored = new HashSet<>();
     final Set<String> allNonStoredDVs = new HashSet<>();
     final Set<String> nonStoredDVsWithoutCopyTargets = new HashSet<>();
+    final Set<String> storedLargeFields = new HashSet<>();
 
     this.fieldInfos = leafReader.getFieldInfos();
-    for (FieldInfo fieldInfo : fieldInfos) {
+    for (FieldInfo fieldInfo : fieldInfos) { // can find materialized dynamic fields, unlike using the Solr IndexSchema.
       final SchemaField schemaField = schema.getFieldOrNull(fieldInfo.name);
-      if (schemaField != null && !schemaField.stored() && schemaField.hasDocValues()) {
+      if (schemaField == null) {
+        continue;
+      }
+      if (!schemaField.stored() && schemaField.hasDocValues()) {
         if (schemaField.useDocValuesAsStored()) {
           nonStoredDVsUsedAsStored.add(fieldInfo.name);
         }
@@ -380,11 +342,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
           nonStoredDVsWithoutCopyTargets.add(fieldInfo.name);
         }
       }
+      if (schemaField.stored() && schemaField.isLarge()) {
+        storedLargeFields.add(schemaField.getName());
+      }
     }
 
     this.nonStoredDVsUsedAsStored = Collections.unmodifiableSet(nonStoredDVsUsedAsStored);
     this.allNonStoredDVs = Collections.unmodifiableSet(allNonStoredDVs);
     this.nonStoredDVsWithoutCopyTargets = Collections.unmodifiableSet(nonStoredDVsWithoutCopyTargets);
+    this.largeFields = Collections.unmodifiableSet(storedLargeFields);
 
     // We already have our own filter cache
     setQueryCache(null);
@@ -677,26 +643,41 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   /*
    * Future optimizations (yonik)
    *
-   * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is begin
+   * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is being
    * retrieved
    */
 
-  /** FieldSelector which loads the specified fields, and loads all other field lazily. */
-  private static class SetNonLazyFieldSelector extends DocumentStoredFieldVisitor {
+  /** {@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. */
+  private class SolrDocumentStoredFieldVisitor extends DocumentStoredFieldVisitor {
     private final Document doc;
-    private final LazyDocument lazyDoc;
+    private final LazyDocument lazyFieldProducer; // arguably a better name than LazyDocument; at least how we use it here
+    private final int docId;
+    private final boolean addLargeFieldsLazily;
 
-    SetNonLazyFieldSelector(Set<String> toLoad, IndexReader reader, int docID) {
+    SolrDocumentStoredFieldVisitor(Set<String> toLoad, IndexReader reader, int docId) {
       super(toLoad);
-      lazyDoc = new LazyDocument(reader, docID);
-      doc = getDocument();
+      this.docId = docId;
+      this.doc = getDocument();
+      this.lazyFieldProducer = toLoad != null && enableLazyFieldLoading ? new LazyDocument(reader, docId) : null;
+      this.addLargeFieldsLazily = (documentCache != null && !largeFields.isEmpty());
+      //TODO can we return Status.STOP after a val is loaded and we know there are no other fields of interest?
+      //    When: toLoad is one single-valued field, no lazyFieldProducer
     }
 
     @Override
     public Status needsField(FieldInfo fieldInfo) throws IOException {
       Status status = super.needsField(fieldInfo);
-      if (status == Status.NO) {
-        doc.add(lazyDoc.getField(fieldInfo));
+      assert status != Status.STOP : "Status.STOP not supported or expected";
+      if (addLargeFieldsLazily && largeFields.contains(fieldInfo.name)) { // load "large" fields using this lazy mechanism
+        if (lazyFieldProducer != null || status == Status.YES) {
+          doc.add(new LargeLazyField(fieldInfo.name, docId));
+        }
+        return Status.NO;
+      }
+      if (status == Status.NO && lazyFieldProducer != null) { // lazy
+        doc.add(lazyFieldProducer.getField(fieldInfo));
       }
       return status;
     }
@@ -717,15 +698,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    * @see IndexReader#document(int, StoredFieldVisitor)
    */
   @Override
-  public void doc(int n, StoredFieldVisitor visitor) throws IOException {
+  public void doc(int docId, StoredFieldVisitor visitor) throws IOException {
     if (documentCache != null) {
-      Document cached = documentCache.get(n);
+      Document cached = documentCache.get(docId);
       if (cached != null) {
         visitFromCached(cached, visitor);
         return;
       }
     }
-    getIndexReader().document(n, visitor);
+    getIndexReader().document(docId, visitor);
   }
 
   /** Executes a stored field visitor against a hit from the document cache */
@@ -735,13 +716,13 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       final Status needsField = visitor.needsField(info);
       if (needsField == Status.STOP) return;
       if (needsField == Status.NO) continue;
-      if (f.binaryValue() != null) {
-        final BytesRef binaryValue = f.binaryValue();
-        final byte copy[] = new byte[binaryValue.length];
-        System.arraycopy(binaryValue.bytes, binaryValue.offset, copy, 0, copy.length);
-        visitor.binaryField(info, copy);
-      } else if (f.numericValue() != null) {
-        final Number numericValue = f.numericValue();
+      BytesRef binaryValue = f.binaryValue();
+      if (binaryValue != null) {
+        visitor.binaryField(info, toByteArrayUnwrapIfPossible(binaryValue));
+        continue;
+      }
+      Number numericValue = f.numericValue();
+      if (numericValue != null) {
         if (numericValue instanceof Double) {
           visitor.doubleField(info, numericValue.doubleValue());
         } else if (numericValue instanceof Integer) {
@@ -753,12 +734,25 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         } else {
           throw new AssertionError();
         }
+        continue;
+      }
+      // must be String
+      if (f instanceof LargeLazyField) { // optimization to avoid premature string conversion
+        visitor.stringField(info, toByteArrayUnwrapIfPossible(((LargeLazyField) f).readBytes()));
       } else {
         visitor.stringField(info, f.stringValue().getBytes(StandardCharsets.UTF_8));
       }
     }
   }
 
+  private byte[] toByteArrayUnwrapIfPossible(BytesRef bytesRef) {
+    if (bytesRef.offset == 0 && bytesRef.bytes.length == bytesRef.length) {
+      return bytesRef.bytes;
+    } else {
+      return Arrays.copyOfRange(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length);
+    }
+  }
+
   /**
    * Retrieve the {@link Document} instance corresponding to the document id.
    * <p>
@@ -775,23 +769,16 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     }
 
     final DirectoryReader reader = getIndexReader();
-    if (fields != null) {
-      if (enableLazyFieldLoading) {
-        final SetNonLazyFieldSelector visitor = new SetNonLazyFieldSelector(fields, reader, i);
-        reader.document(i, visitor);
-        d = visitor.doc;
-      } else if (documentCache == null) {
-        d = reader.document(i, fields);
-      } else {
-        // we do not pass the fields in this case because that would return an incomplete document which would
-        // be eventually cached. The alternative would be to read the stored fields twice; once with the fields
-        // and then without for caching leading to a performance hit
-        // see SOLR-8858 for related discussion
-        d = reader.document(i);
-      }
-    } else {
-      d = reader.document(i);
+    if (documentCache != null && !enableLazyFieldLoading) {
+      // we do not filter the fields in this case because that would return an incomplete document which would
+      // be eventually cached. The alternative would be to read the stored fields twice; once with the fields
+      // and then without for caching leading to a performance hit
+      // see SOLR-8858 for related discussion
+      fields = null;
     }
+    final SolrDocumentStoredFieldVisitor visitor = new SolrDocumentStoredFieldVisitor(fields, reader, i);
+    reader.document(i, visitor);
+    d = visitor.getDocument();
 
     if (documentCache != null) {
       documentCache.put(i, d);
@@ -800,6 +787,103 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return d;
   }
 
+  /** Unlike LazyDocument.LazyField, we (a) don't cache large values, and (b) provide access to the byte[]. */
+  class LargeLazyField implements IndexableField {
+
+    final String name;
+    final int docId;
+    // synchronize on 'this' to access:
+    BytesRef cachedBytes; // we only conditionally populate this if it's big enough
+
+    private LargeLazyField(String name, int docId) {
+      this.name = name;
+      this.docId = docId;
+    }
+
+    @Override
+    public String toString() {
+      return fieldType().toString() + "<" + name() + ">"; // mimic Field.java
+    }
+
+    @Override
+    public String name() {
+      return name;
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      return schema.getField(name());
+    }
+
+    @Override
+    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;
+    }
+
+    @Override
+    public synchronized String stringValue() {
+      try {
+        return readBytes().utf8ToString();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    synchronized BytesRef readBytes() throws IOException {
+      if (cachedBytes != null) {
+        return cachedBytes;
+      } else {
+        BytesRef bytesRef = new BytesRef();
+        getIndexReader().document(docId, new StoredFieldVisitor() {
+          boolean done = false;
+          @Override
+          public Status needsField(FieldInfo fieldInfo) throws IOException {
+            if (done) {
+              return Status.STOP;
+            }
+            return fieldInfo.name.equals(name()) ? Status.YES : Status.NO;
+          }
+
+          @Override
+          public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException {
+            bytesRef.bytes = value;
+            bytesRef.length = value.length;
+            done = true;
+          }
+
+          @Override
+          public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
+            throw new UnsupportedOperationException("'large' binary fields are not (yet) supported");
+          }
+        });
+        if (bytesRef.length < largeValueLengthCacheThreshold) {
+          return cachedBytes = bytesRef;
+        } else {
+          return bytesRef;
+        }
+      }
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return null;
+    }
+
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+  }
+
   /**
    * This will fetch and add the docValues fields to a given SolrDocument/SolrInputDocument
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
index abf4a1f..e3d2011 100644
--- a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
+++ b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
@@ -20,7 +20,6 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.solr.common.SolrException;
@@ -55,7 +54,7 @@ public class DocumentBuilder {
         assert val instanceof NumericDocValuesField: "Expected in-place update to be done on"
             + " NDV fields only.";
       }
-      doc.add((Field)val);
+      doc.add((IndexableField)val);
       return;
     }
     for (IndexableField f : field.getType().createFields(field, val)) {
@@ -69,10 +68,10 @@ public class DocumentBuilder {
         //    assert f instanceof NumericDocValuesField
         if (forInPlaceUpdate) {
           if (f instanceof NumericDocValuesField) {
-            doc.add((Field) f);
+            doc.add(f);
           }
         } else {
-          doc.add((Field) f);
+          doc.add(f);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml b/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml
index ab18354..8d71088 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml
@@ -17,7 +17,7 @@
 
 <!-- Test schema file for UnifiedHighlighter -->
 
-<schema name="unifiedhighlight" version="1.0">
+<schema name="unifiedhighlight" version="1.6" >
   <fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
 
   <!-- basic text field: no offsets! -->
@@ -39,7 +39,7 @@
   <field name="id" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
   <field name="text" type="text_offsets" indexed="true" stored="true"/>
   <field name="text2" type="text" indexed="true" stored="true"/>
-  <field name="text3" type="text_offsets" indexed="true" stored="true"/>
+  <field name="text3" type="text_offsets" indexed="true" stored="true"         large="true"/>
 
   <defaultSearchField>text</defaultSearchField>
   <uniqueKey>id</uniqueKey>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
index abd4fbe..26224ad 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
@@ -24,7 +24,7 @@
 
   <schemaFactory class="ManagedIndexSchemaFactory">
     <bool name="mutable">${managed.schema.mutable}</bool>
-    <str name="managedSchemaResourceName">managed-schema</str>
+    <str name="managedSchemaResourceName">${managed.schema.resourceName:managed-schema}</str>
   </schemaFactory>
 
   <codecFactory class="solr.SchemaCodecFactory"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
index 966d8ef..140fd7e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java
@@ -160,7 +160,7 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase {
   }
 
   /** 
-   * Tests thta all TransformerFactories that are implicitly provided by Solr are tested in this class
+   * Tests that all TransformerFactories that are implicitly provided by Solr are tested in this class
    *
    * @see FlValidator#getDefaultTransformerFactoryName
    * @see #FL_VALIDATORS

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index 9835518..2f7a003 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -19,6 +19,7 @@ package org.apache.solr.highlight;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 /** Tests for the UnifiedHighlighter Solr plugin **/
@@ -26,7 +27,10 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
   
   @BeforeClass
   public static void beforeClass() throws Exception {
-    initCore("solrconfig-basic.xml", "schema-unifiedhighlight.xml");
+    System.setProperty("filterCache.enabled", "false");
+    System.setProperty("queryResultCache.enabled", "false");
+    System.setProperty("documentCache.enabled", "true"); // this is why we use this particular solrconfig
+    initCore("solrconfig-cache-enable-disable.xml", "schema-unifiedhighlight.xml");
     
     // test our config is sane, just to be sure:
 
@@ -36,6 +40,12 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
     assertTrue(schema.getField("text3").storeOffsetsWithPositions());
     assertFalse(schema.getField("text2").storeOffsetsWithPositions());
   }
+  @AfterClass
+  public static void afterClass() {
+    System.clearProperty("filterCache.enabled");
+    System.clearProperty("queryResultCache.enabled");
+    System.clearProperty("documentCache.enabled");
+  }
   
   @Override
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java b/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java
index b8ed296..90c92d0 100644
--- a/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java
+++ b/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java
@@ -35,8 +35,7 @@ import org.apache.lucene.util.BytesRef;
 public class SortableBinaryField extends BinaryField {
 
   @Override
-  public void checkSchemaField(final SchemaField field) {
-    // NOOP, It's Aaaaaall Good.
+  protected void checkSupportsDocValues() { // we support DocValues
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
new file mode 100644
index 0000000..d05c69c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.LazyDocument;
+import org.apache.lucene.index.IndexableField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.request.SolrQueryRequestBase;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.util.RefCounted;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class LargeFieldTest extends SolrTestCaseJ4 {
+
+  private static final String ID_FLD = "str"; // TODO alter underlying schema to be 'id'
+  private static final String LAZY_FIELD = "lazyField";
+  private static final String BIG_FIELD = "bigField";
+
+  @BeforeClass
+  public static void initManagedSchemaCore() throws Exception {
+    // TODO propose convenience API for this?
+    // This testing approach means no new solrconfig or 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?");
+
+    // modify solr config  TODO propose more convenient API for this; maybe with JSON-ification of a map
+    try (SolrQueryRequestBase req = (SolrQueryRequestBase) req()) {
+      req.getContext().put("httpMethod", "POST");
+      req.setContentStreams(Collections.singleton(new ContentStreamBase.StringStream(
+          "{ 'set-property':{" +
+              "'query.enableLazyFieldLoading':true, " +
+              "'query.documentCache.class':'solr.LRUCache'" +
+              "}}"
+      )));
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      h.getCore().execute(h.getCore().getRequestHandler("/config"), req, rsp);
+      assertNull(rsp.getException());
+    }
+
+    boolean PERSIST_FALSE = false; // don't write to test resource dir
+
+    IndexSchema schema = h.getCore().getLatestSchema();
+    schema = schema.addFieldTypes(Collections.singletonList(
+        schema.newFieldType("textType", "solr.TextField", // redundant; TODO improve api
+            map("name", "textType",   "class", "solr.TextField",
+                "analyzer", map("class", "org.apache.lucene.analysis.standard.StandardAnalyzer")))),
+        PERSIST_FALSE);
+    schema = schema.addFields(Arrays.asList(
+        schema.newField(LAZY_FIELD, "textType", map()),
+        schema.newField(BIG_FIELD, "textType", map("large", true))),
+        Collections.emptyMap(),
+        PERSIST_FALSE);
+
+    h.getCore().setLatestSchema(schema);
+  }
+
+  @Test
+  public void test() throws Exception {
+    // add just one document (docid 0)
+    assertU(adoc(ID_FLD, "101", LAZY_FIELD, "lzy", BIG_FIELD, "big document field one"));
+    assertU(commit());
+
+    // trigger the ID_FLD to get into the doc cache; don't reference other fields
+    assertQ(req("q", "101", "df", ID_FLD, "fl", ID_FLD)); // eager load ID_FLD; rest are lazy
+
+    // fetch the document; we know it will be from the documentCache, docId 0
+    final Document d;
+    RefCounted<SolrIndexSearcher> searcherRef = h.getCore().getSearcher();
+    try {
+      d = searcherRef.get().doc(0);
+    } finally {
+      searcherRef.decref();
+    }
+
+    assertEager(d, ID_FLD);
+    assertLazyNotLoaded(d, LAZY_FIELD);
+    assertLazyNotLoaded(d, BIG_FIELD);
+
+    assertQ(req("q", "101", "df", ID_FLD, "fl", LAZY_FIELD)); // trigger load of LAZY_FIELD
+
+    assertEager(d, ID_FLD);
+    assertLazyLoaded(d, LAZY_FIELD); // loaded now
+    assertLazyNotLoaded(d, BIG_FIELD); // because big fields are handled separately
+
+    assertQ(req("q", "101", "df", ID_FLD, "fl", BIG_FIELD)); // trigger load of BIG_FIELD
+
+    assertEager(d, ID_FLD);
+    assertLazyLoaded(d, LAZY_FIELD);
+    assertLazyLoaded(d, BIG_FIELD); // loaded now
+  }
+
+  private void assertEager(Document d, String fieldName) {
+    assertFalse( d.getField(fieldName) instanceof LazyDocument.LazyField);
+  }
+
+  private void assertLazyNotLoaded(Document d, String fieldName) {
+    IndexableField field = d.getField(fieldName);
+    if (fieldName == BIG_FIELD) {
+      assertTrue(field instanceof SolrIndexSearcher.LargeLazyField);
+      assertFalse(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded());
+    } else {
+      assertTrue(field instanceof LazyDocument.LazyField);
+      assertFalse(((LazyDocument.LazyField)field).hasBeenLoaded());
+    }
+  }
+
+  private void assertLazyLoaded(Document d, String fieldName) {
+    IndexableField field = d.getField(fieldName);
+    if (fieldName == BIG_FIELD) {
+      assertTrue(field instanceof SolrIndexSearcher.LargeLazyField);
+      assertTrue(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded());
+    } else {
+      assertTrue(field instanceof LazyDocument.LazyField);
+      assertTrue(((LazyDocument.LazyField)field).hasBeenLoaded());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2502af9f/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 825e7c7..04fc3ff 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -852,7 +852,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   /** Validates a query matches some XPath test expressions and closes the query */
   public static void assertQ(String message, SolrQueryRequest req, String... tests) {
     try {
-      String m = (null == message) ? "" : message + " ";
+      String m = (null == message) ? "" : message + " "; // TODO log 'm' !!!
       String response = h.query(req);
 
       if (req.getParams().getBool("facet", false)) {